Merge remote-tracking branch 'upstream/master' into HEAD

This commit is contained in:
Anton Popov 2024-01-06 20:58:17 +00:00
commit 8c9cf394b3
125 changed files with 1448 additions and 1290 deletions

View File

@ -143,9 +143,17 @@ public:
return alias;
}
const String & getOriginalAlias() const
{
return original_alias.empty() ? alias : original_alias;
}
/// Set node alias
void setAlias(String alias_value)
{
if (original_alias.empty())
original_alias = std::move(alias);
alias = std::move(alias_value);
}
@ -276,6 +284,9 @@ protected:
private:
String alias;
/// An alias from query. Alias can be replaced by query passes,
/// but we need to keep the original one to support additional_table_filters.
String original_alias;
ASTPtr original_ast;
};

View File

@ -52,6 +52,7 @@
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include <Analyzer/createUniqueTableAliases.h>
#include <Analyzer/Utils.h>
#include <Analyzer/SetUtils.h>
#include <Analyzer/AggregationUtils.h>
@ -1198,7 +1199,7 @@ private:
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope);
@ -2168,7 +2169,12 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_
scope.scope_node->formatASTForErrorMessage());
--positional_argument_number;
*node_to_replace = projection_nodes[positional_argument_number];
*node_to_replace = projection_nodes[positional_argument_number]->clone();
if (auto it = resolved_expressions.find(projection_nodes[positional_argument_number]);
it != resolved_expressions.end())
{
resolved_expressions[*node_to_replace] = it->second;
}
}
}
@ -7366,6 +7372,7 @@ void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context
{
QueryAnalyzer analyzer;
analyzer.resolve(query_tree_node, table_expression, context);
createUniqueTableAliases(query_tree_node, table_expression, context);
}
}

View File

@ -326,7 +326,7 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
}
}
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
{
QueryTreeNodes result;
@ -357,6 +357,8 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node)
{
auto & array_join_node = node_to_process->as<ArrayJoinNode &>();
nodes_to_process.push_front(array_join_node.getTableExpression());
if (add_array_join)
result.push_back(std::move(node_to_process));
break;
}
case QueryTreeNodeType::JOIN:

View File

@ -51,7 +51,7 @@ std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
/// Extract table, table function, query, union from join tree
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node);
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
/// Extract left table expression from join tree
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);

View File

@ -0,0 +1,141 @@
#include <memory>
#include <unordered_map>
#include <Analyzer/createUniqueTableAliases.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/LambdaNode.h>
#include <Analyzer/Utils.h>
namespace DB
{
namespace
{
class CreateUniqueTableAliasesVisitor : public InDepthQueryTreeVisitorWithContext<CreateUniqueTableAliasesVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<CreateUniqueTableAliasesVisitor>;
explicit CreateUniqueTableAliasesVisitor(const ContextPtr & context)
: Base(context)
{
// Insert a fake node on top of the stack.
scope_nodes_stack.push_back(std::make_shared<LambdaNode>(Names{}, nullptr));
}
void enterImpl(QueryTreeNodePtr & node)
{
auto node_type = node->getNodeType();
switch (node_type)
{
case QueryTreeNodeType::QUERY:
[[fallthrough]];
case QueryTreeNodeType::UNION:
{
/// Queries like `(SELECT 1) as t` have invalid syntax. To avoid creating such queries (e.g. in StorageDistributed)
/// we need to remove aliases for top level queries.
/// N.B. Subquery depth starts count from 1, so the following condition checks if it's a top level.
if (getSubqueryDepth() == 1)
{
node->removeAlias();
break;
}
[[fallthrough]];
}
case QueryTreeNodeType::TABLE:
[[fallthrough]];
case QueryTreeNodeType::TABLE_FUNCTION:
[[fallthrough]];
case QueryTreeNodeType::ARRAY_JOIN:
{
auto & alias = table_expression_to_alias[node];
if (alias.empty())
{
scope_to_nodes_with_aliases[scope_nodes_stack.back()].push_back(node);
alias = fmt::format("__table{}", ++next_id);
node->setAlias(alias);
}
break;
}
default:
break;
}
switch (node_type)
{
case QueryTreeNodeType::QUERY:
[[fallthrough]];
case QueryTreeNodeType::UNION:
[[fallthrough]];
case QueryTreeNodeType::LAMBDA:
scope_nodes_stack.push_back(node);
break;
default:
break;
}
}
void leaveImpl(QueryTreeNodePtr & node)
{
if (scope_nodes_stack.back() == node)
{
if (auto it = scope_to_nodes_with_aliases.find(scope_nodes_stack.back());
it != scope_to_nodes_with_aliases.end())
{
for (const auto & node_with_alias : it->second)
{
table_expression_to_alias.erase(node_with_alias);
}
scope_to_nodes_with_aliases.erase(it);
}
scope_nodes_stack.pop_back();
}
/// Here we revisit subquery for IN function. Reasons:
/// * For remote query execution, query tree may be traversed a few times.
/// In such a case, it is possible to get AST like
/// `IN ((SELECT ... FROM table AS __table4) AS __table1)` which result in
/// `Multiple expressions for the alias` exception
/// * Tables in subqueries could have different aliases => different three hashes,
/// which is important to be able to find a set in PreparedSets
/// See 01253_subquery_in_aggregate_function_JustStranger.
///
/// So, we revisit this subquery to make aliases stable.
/// This should be safe cause columns from IN subquery can't be used in main query anyway.
if (node->getNodeType() == QueryTreeNodeType::FUNCTION)
{
auto * function_node = node->as<FunctionNode>();
if (isNameOfInFunction(function_node->getFunctionName()))
{
auto arg = function_node->getArguments().getNodes().back();
/// Avoid aliasing IN `table`
if (arg->getNodeType() != QueryTreeNodeType::TABLE)
CreateUniqueTableAliasesVisitor(getContext()).visit(function_node->getArguments().getNodes().back());
}
}
}
private:
size_t next_id = 0;
// Stack of nodes which create scopes: QUERY, UNION and LAMBDA.
QueryTreeNodes scope_nodes_stack;
std::unordered_map<QueryTreeNodePtr, QueryTreeNodes> scope_to_nodes_with_aliases;
// We need to use raw pointer as a key, not a QueryTreeNodePtrWithHash.
std::unordered_map<QueryTreeNodePtr, String> table_expression_to_alias;
};
}
void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & /*table_expression*/, const ContextPtr & context)
{
CreateUniqueTableAliasesVisitor(context).visit(node);
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <memory>
#include <Interpreters/Context_fwd.h>
class IQueryTreeNode;
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
namespace DB
{
/*
* For each table expression in the Query Tree generate and add a unique alias.
* If table expression had an alias in initial query tree, override it.
*/
void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, const ContextPtr & context);
}

View File

@ -157,7 +157,7 @@ class IColumn;
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
@ -709,7 +709,6 @@ class IColumn;
M(Bool, query_plan_execute_functions_after_sorting, true, "Allow to re-order functions after sorting", 0) \
M(Bool, query_plan_reuse_storage_ordering_for_window_functions, true, "Allow to use the storage sorting for window functions", 0) \
M(Bool, query_plan_lift_up_union, true, "Allow to move UNIONs up so that more parts of the query plan can be optimized", 0) \
M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \
M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimization", 0) \
M(Bool, query_plan_aggregation_in_order, true, "Use query plan for aggregation-in-order optimization", 0) \
M(Bool, query_plan_remove_redundant_sorting, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries", 0) \
@ -918,6 +917,7 @@ class IColumn;
MAKE_OBSOLETE(M, Bool, optimize_move_functions_out_of_any, false) \
MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \
MAKE_OBSOLETE(M, Bool, query_plan_optimize_primary_key, true) \
/** The section above is for obsolete settings. Do not add anything there. */

View File

@ -1419,7 +1419,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool
return set;
}
FutureSetPtr external_table_set;
FutureSetFromSubqueryPtr external_table_set;
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).

View File

@ -664,26 +664,26 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
for (size_t i = 0; i < aggregate_functions.size(); ++i)
{
const auto * function = aggregate_functions[i];
bool function_is_compilable = function->isCompilable();
if (!function_is_compilable)
continue;
size_t offset_of_aggregate_function = offsets_of_aggregate_states[i];
AggregateFunctionWithOffset function_to_compile
if (function->isCompilable())
{
.function = function,
.aggregate_data_offset = offset_of_aggregate_function
};
AggregateFunctionWithOffset function_to_compile
{
.function = function,
.aggregate_data_offset = offset_of_aggregate_function
};
functions_to_compile.emplace_back(std::move(function_to_compile));
functions_to_compile.emplace_back(std::move(function_to_compile));
functions_description += function->getDescription();
functions_description += ' ';
functions_description += function->getDescription();
functions_description += ' ';
functions_description += std::to_string(offset_of_aggregate_function);
functions_description += ' ';
functions_description += std::to_string(offset_of_aggregate_function);
functions_description += ' ';
}
is_aggregate_function_compiled[i] = true;
is_aggregate_function_compiled[i] = function->isCompilable();
}
if (functions_to_compile.empty())
@ -1685,13 +1685,14 @@ bool Aggregator::executeOnBlock(Columns columns,
/// For the case when there are no keys (all aggregate into one row).
if (result.type == AggregatedDataVariants::Type::without_key)
{
#if USE_EMBEDDED_COMPILER
if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_functions_instructions.data()))
{
executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}
else
#endif
/// TODO: Enable compilation after investigation
// #if USE_EMBEDDED_COMPILER
// if (compiled_aggregate_functions_holder)
// {
// executeWithoutKeyImpl<true>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
// }
// else
// #endif
{
executeWithoutKeyImpl<false>(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool);
}

View File

@ -82,8 +82,8 @@ private:
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
class FutureSet;
using FutureSetPtr = std::shared_ptr<FutureSet>;
class FutureSetFromSubquery;
using FutureSetFromSubqueryPtr = std::shared_ptr<FutureSetFromSubquery>;
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
/// Such table can be accessed from everywhere by its ID.
@ -116,7 +116,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
IDatabase * temporary_tables = nullptr;
UUID id = UUIDHelpers::Nil;
FutureSetPtr future_set;
FutureSetFromSubqueryPtr future_set;
};
///TODO maybe remove shared_ptr from here?

View File

@ -2315,12 +2315,25 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
else
{
// It's possible to optimize count() given only partition predicates
SelectQueryInfo temp_query_info;
temp_query_info.query = query_ptr;
temp_query_info.syntax_analyzer_result = syntax_analyzer_result;
temp_query_info.prepared_sets = query_analyzer->getPreparedSets();
ActionsDAG::NodeRawConstPtrs filter_nodes;
if (analysis_result.hasPrewhere())
{
auto & prewhere_info = analysis_result.prewhere_info;
filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name));
return storage->totalRowsByPartitionPredicate(temp_query_info, context);
if (prewhere_info->row_level_filter)
filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name));
}
if (analysis_result.hasWhere())
{
filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name));
}
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes, {}, context);
if (!filter_actions_dag)
return {};
return storage->totalRowsByPartitionPredicate(filter_actions_dag, context);
}
}

View File

@ -67,8 +67,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
{
const auto & function_argument_types = function.getArgumentTypes();
auto & context = module.getContext();
llvm::IRBuilder<> b(context);
llvm::IRBuilder<> b(module.getContext());
auto * size_type = b.getIntNTy(sizeof(size_t) * 8);
auto * data_type = llvm::StructType::get(b.getInt8PtrTy(), b.getInt8PtrTy());
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { size_type, data_type->getPointerTo() }, /*isVarArg=*/false);
@ -76,8 +75,6 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
/// Create function in module
auto * func = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, function.getName(), module);
func->setAttributes(llvm::AttributeList::get(context, {{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * args = func->args().begin();
llvm::Value * rows_count_arg = args++;
llvm::Value * columns_arg = args++;
@ -199,9 +196,6 @@ static void compileCreateAggregateStatesFunctions(llvm::Module & module, const s
auto * create_aggregate_states_function_type = llvm::FunctionType::get(b.getVoidTy(), { aggregate_data_places_type }, false);
auto * create_aggregate_states_function = llvm::Function::Create(create_aggregate_states_function_type, llvm::Function::ExternalLinkage, name, module);
create_aggregate_states_function->setAttributes(
llvm::AttributeList::get(context, {{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = create_aggregate_states_function->args().begin();
llvm::Value * aggregate_data_place_arg = arguments++;
@ -247,11 +241,6 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
auto * add_into_aggregate_states_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), places_type }, false);
auto * add_into_aggregate_states_func = llvm::Function::Create(add_into_aggregate_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
add_into_aggregate_states_func->setAttributes(llvm::AttributeList::get(
context,
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = add_into_aggregate_states_func->args().begin();
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
@ -307,7 +296,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
llvm::Value * aggregation_place = nullptr;
if (places_argument_type == AddIntoAggregateStatesPlacesArgumentType::MultiplePlaces)
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
aggregation_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), places_arg, counter_phi));
else
aggregation_place = places_arg;
@ -324,7 +313,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
auto & column = columns[previous_columns_size + column_argument_index];
const auto & argument_type = arguments_types[column_argument_index];
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateInBoundsGEP(column.data_element_type, column.data_ptr, counter_phi));
auto * column_data_element = b.CreateLoad(column.data_element_type, b.CreateGEP(column.data_element_type, column.data_ptr, counter_phi));
if (!argument_type->isNullable())
{
@ -332,7 +321,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
continue;
}
auto * column_null_data_with_offset = b.CreateInBoundsGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
auto * column_null_data_with_offset = b.CreateGEP(b.getInt8Ty(), column.null_data_ptr, counter_phi);
auto * is_null = b.CreateICmpNE(b.CreateLoad(b.getInt8Ty(), column_null_data_with_offset), b.getInt8(0));
auto * nullable_unitialized = llvm::Constant::getNullValue(toNullableType(b, column.data_element_type));
auto * first_insert = b.CreateInsertValue(nullable_unitialized, column_data_element, {0});
@ -365,8 +354,7 @@ static void compileAddIntoAggregateStatesFunctions(llvm::Module & module,
static void compileMergeAggregatesStates(llvm::Module & module, const std::vector<AggregateFunctionWithOffset> & functions, const std::string & name)
{
auto & context = module.getContext();
llvm::IRBuilder<> b(context);
llvm::IRBuilder<> b(module.getContext());
auto * aggregate_data_place_type = b.getInt8Ty()->getPointerTo();
auto * aggregate_data_places_type = aggregate_data_place_type->getPointerTo();
@ -377,11 +365,6 @@ static void compileMergeAggregatesStates(llvm::Module & module, const std::vecto
auto * merge_aggregates_states_func
= llvm::Function::Create(merge_aggregates_states_func_declaration, llvm::Function::ExternalLinkage, name, module);
merge_aggregates_states_func->setAttributes(llvm::AttributeList::get(
context,
{{1, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{2, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = merge_aggregates_states_func->args().begin();
llvm::Value * aggregate_data_places_dst_arg = arguments++;
llvm::Value * aggregate_data_places_src_arg = arguments++;
@ -443,11 +426,6 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
auto * insert_aggregates_into_result_func_declaration = llvm::FunctionType::get(b.getVoidTy(), { size_type, size_type, column_type->getPointerTo(), aggregate_data_places_type }, false);
auto * insert_aggregates_into_result_func = llvm::Function::Create(insert_aggregates_into_result_func_declaration, llvm::Function::ExternalLinkage, name, module);
insert_aggregates_into_result_func->setAttributes(llvm::AttributeList::get(
context,
{{3, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)},
{4, llvm::Attribute::get(context, llvm::Attribute::AttrKind::NoAlias)}}));
auto * arguments = insert_aggregates_into_result_func->args().begin();
llvm::Value * row_start_arg = arguments++;
llvm::Value * row_end_arg = arguments++;
@ -482,7 +460,7 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
auto * counter_phi = b.CreatePHI(row_start_arg->getType(), 2);
counter_phi->addIncoming(row_start_arg, entry);
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateInBoundsGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
auto * aggregate_data_place = b.CreateLoad(b.getInt8Ty()->getPointerTo(), b.CreateGEP(b.getInt8Ty()->getPointerTo(), aggregate_data_places_arg, counter_phi));
for (size_t i = 0; i < functions.size(); ++i)
{
@ -492,11 +470,11 @@ static void compileInsertAggregatesIntoResultColumns(llvm::Module & module, cons
const auto * aggregate_function_ptr = functions[i].function;
auto * final_value = aggregate_function_ptr->compileGetResult(b, aggregation_place_with_offset);
auto * result_column_data_element = b.CreateInBoundsGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
auto * result_column_data_element = b.CreateGEP(columns[i].data_element_type, columns[i].data_ptr, counter_phi);
if (columns[i].null_data_ptr)
{
b.CreateStore(b.CreateExtractValue(final_value, {0}), result_column_data_element);
auto * result_column_is_null_element = b.CreateInBoundsGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
auto * result_column_is_null_element = b.CreateGEP(b.getInt8Ty(), columns[i].null_data_ptr, counter_phi);
b.CreateStore(b.CreateSelect(b.CreateExtractValue(final_value, {1}), b.getInt8(1), b.getInt8(0)), result_column_is_null_element);
}
else

View File

@ -97,7 +97,7 @@ FutureSetFromSubquery::FutureSetFromSubquery(
String key,
std::unique_ptr<QueryPlan> source_,
StoragePtr external_table_,
FutureSetPtr external_table_set_,
std::shared_ptr<FutureSetFromSubquery> external_table_set_,
const Settings & settings,
bool in_subquery_)
: external_table(std::move(external_table_))
@ -168,6 +168,24 @@ std::unique_ptr<QueryPlan> FutureSetFromSubquery::build(const ContextPtr & conte
return plan;
}
void FutureSetFromSubquery::buildSetInplace(const ContextPtr & context)
{
if (external_table_set)
external_table_set->buildSetInplace(context);
auto plan = build(context);
if (!plan)
return;
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
pipeline.complete(std::make_shared<EmptySink>(Block()));
CompletedPipelineExecutor executor(pipeline);
executor.execute();
}
SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
{
if (!context->getSettingsRef().use_index_for_in_with_subqueries)
@ -233,7 +251,7 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes &
return buf.str();
}
FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings)
FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings)
{
auto from_tuple = std::make_shared<FutureSetFromTuple>(std::move(block), settings);
const auto & set_types = from_tuple->getTypes();
@ -247,7 +265,7 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set
return from_tuple;
}
FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
{
auto from_storage = std::make_shared<FutureSetFromStorage>(std::move(set_));
auto [it, inserted] = sets_from_storage.emplace(key, from_storage);
@ -258,11 +276,11 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
return from_storage;
}
FutureSetPtr PreparedSets::addFromSubquery(
FutureSetFromSubqueryPtr PreparedSets::addFromSubquery(
const Hash & key,
std::unique_ptr<QueryPlan> source,
StoragePtr external_table,
FutureSetPtr external_table_set,
FutureSetFromSubqueryPtr external_table_set,
const Settings & settings,
bool in_subquery)
{
@ -282,7 +300,7 @@ FutureSetPtr PreparedSets::addFromSubquery(
return from_subquery;
}
FutureSetPtr PreparedSets::addFromSubquery(
FutureSetFromSubqueryPtr PreparedSets::addFromSubquery(
const Hash & key,
QueryTreeNodePtr query_tree,
const Settings & settings)
@ -300,7 +318,7 @@ FutureSetPtr PreparedSets::addFromSubquery(
return from_subquery;
}
FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const
FutureSetFromTuplePtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const
{
auto it = sets_from_tuple.find(key);
if (it == sets_from_tuple.end())

View File

@ -69,6 +69,8 @@ private:
SetPtr set;
};
using FutureSetFromStoragePtr = std::shared_ptr<FutureSetFromStorage>;
/// Set from tuple is filled as well as set from storage.
/// Additionally, it can be converted to set useful for PK.
class FutureSetFromTuple final : public FutureSet
@ -86,6 +88,8 @@ private:
SetKeyColumns set_key_columns;
};
using FutureSetFromTuplePtr = std::shared_ptr<FutureSetFromTuple>;
/// Set from subquery can be built inplace for PK or in CreatingSet step.
/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created,
/// but ordinary set would be created instead.
@ -96,7 +100,7 @@ public:
String key,
std::unique_ptr<QueryPlan> source_,
StoragePtr external_table_,
FutureSetPtr external_table_set_,
std::shared_ptr<FutureSetFromSubquery> external_table_set_,
const Settings & settings,
bool in_subquery_);
@ -110,6 +114,7 @@ public:
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
std::unique_ptr<QueryPlan> build(const ContextPtr & context);
void buildSetInplace(const ContextPtr & context);
QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); }
void setQueryPlan(std::unique_ptr<QueryPlan> source_);
@ -119,7 +124,7 @@ public:
private:
SetAndKeyPtr set_and_key;
StoragePtr external_table;
FutureSetPtr external_table_set;
std::shared_ptr<FutureSetFromSubquery> external_table_set;
std::unique_ptr<QueryPlan> source;
QueryTreeNodePtr query_tree;
@ -130,6 +135,8 @@ private:
// with new analyzer it's not a case
};
using FutureSetFromSubqueryPtr = std::shared_ptr<FutureSetFromSubquery>;
/// Container for all the sets used in query.
class PreparedSets
{
@ -141,32 +148,32 @@ public:
UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; }
};
using SetsFromTuple = std::unordered_map<Hash, std::vector<std::shared_ptr<FutureSetFromTuple>>, Hashing>;
using SetsFromStorage = std::unordered_map<Hash, std::shared_ptr<FutureSetFromStorage>, Hashing>;
using SetsFromSubqueries = std::unordered_map<Hash, std::shared_ptr<FutureSetFromSubquery>, Hashing>;
using SetsFromTuple = std::unordered_map<Hash, std::vector<FutureSetFromTuplePtr>, Hashing>;
using SetsFromStorage = std::unordered_map<Hash, FutureSetFromStoragePtr, Hashing>;
using SetsFromSubqueries = std::unordered_map<Hash, FutureSetFromSubqueryPtr, Hashing>;
FutureSetPtr addFromStorage(const Hash & key, SetPtr set_);
FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings);
FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_);
FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings);
FutureSetPtr addFromSubquery(
FutureSetFromSubqueryPtr addFromSubquery(
const Hash & key,
std::unique_ptr<QueryPlan> source,
StoragePtr external_table,
FutureSetPtr external_table_set,
FutureSetFromSubqueryPtr external_table_set,
const Settings & settings,
bool in_subquery = false);
FutureSetPtr addFromSubquery(
FutureSetFromSubqueryPtr addFromSubquery(
const Hash & key,
QueryTreeNodePtr query_tree,
const Settings & settings);
FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const;
std::shared_ptr<FutureSetFromStorage> findStorage(const Hash & key) const;
std::shared_ptr<FutureSetFromSubquery> findSubquery(const Hash & key) const;
FutureSetFromTuplePtr findTuple(const Hash & key, const DataTypes & types) const;
FutureSetFromStoragePtr findStorage(const Hash & key) const;
FutureSetFromSubqueryPtr findSubquery(const Hash & key) const;
void markAsINSubquery(const Hash & key);
using Subqueries = std::vector<std::shared_ptr<FutureSetFromSubquery>>;
using Subqueries = std::vector<FutureSetFromSubqueryPtr>;
Subqueries getSubqueries() const;
bool hasSubqueries() const { return !sets_from_subqueries.empty(); }

View File

@ -36,7 +36,6 @@ struct RequiredSourceColumnsData
bool has_table_join = false;
bool has_array_join = false;
bool visit_index_hint = false;
bool addColumnAliasIfAny(const IAST & ast);
void addColumnIdentifier(const ASTIdentifier & node);

View File

@ -72,11 +72,6 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data)
}
if (auto * t = ast->as<ASTFunction>())
{
/// "indexHint" is a special function for index analysis.
/// Everything that is inside it is not calculated. See KeyCondition
if (!data.visit_index_hint && t->name == "indexHint")
return;
data.addColumnAliasIfAny(*ast);
visit(*t, ast, data);
return;

View File

@ -995,13 +995,12 @@ void TreeRewriterResult::collectSourceColumns(bool add_special)
/// Calculate which columns are required to execute the expression.
/// Then, delete all other columns from the list of available columns.
/// After execution, columns will only contain the list of columns needed to read from the table.
bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw)
bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw)
{
/// We calculate required_source_columns with source_columns modifications and swap them on exit
required_source_columns = source_columns;
RequiredSourceColumnsVisitor::Data columns_context;
columns_context.visit_index_hint = visit_index_hint;
RequiredSourceColumnsVisitor(columns_context).visit(query);
NameSet source_column_names;
@ -1385,7 +1384,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
result.window_function_asts = getWindowFunctions(query, *select_query);
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
result.collectUsedColumns(query, true);
if (!result.missed_subcolumns.empty())
{
@ -1422,7 +1421,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
result.aggregates = getAggregates(query, *select_query);
result.window_function_asts = getWindowFunctions(query, *select_query);
result.expressions_with_window_function = getExpressionsWithWindowFunctions(query);
result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key);
result.collectUsedColumns(query, true);
}
}
@ -1499,7 +1498,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
else
assertNoAggregates(query, "in wrong place");
bool is_ok = result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key, no_throw);
bool is_ok = result.collectUsedColumns(query, false, no_throw);
if (!is_ok)
return {};

View File

@ -88,7 +88,7 @@ struct TreeRewriterResult
bool add_special = true);
void collectSourceColumns(bool add_special);
bool collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint, bool no_throw = false);
bool collectUsedColumns(const ASTPtr & query, bool is_select, bool no_throw = false);
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; }
NameSet getArrayJoinSourceNameSet() const;

View File

@ -1057,7 +1057,7 @@ void addBuildSubqueriesForSetsStepIfNeeded(
Planner subquery_planner(
query_tree,
subquery_options,
planner_context->getGlobalPlannerContext());
std::make_shared<GlobalPlannerContext>()); //planner_context->getGlobalPlannerContext());
subquery_planner.buildQueryPlanIfNeeded();
subquery->setQueryPlan(std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan()));

View File

@ -20,12 +20,15 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Quer
return createColumnIdentifier(column_node_typed.getColumn(), column_source_node);
}
const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & /*column_source_node*/)
const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node)
{
std::string column_identifier;
column_identifier += column.name;
column_identifier += '_' + std::to_string(column_identifiers.size());
const auto & source_alias = column_source_node->getAlias();
if (!source_alias.empty())
column_identifier = source_alias + "." + column.name;
else
column_identifier = column.name;
auto [it, inserted] = column_identifiers.emplace(column_identifier);
assert(inserted);

View File

@ -817,7 +817,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
}
const auto & table_expression_alias = table_expression->getAlias();
const auto & table_expression_alias = table_expression->getOriginalAlias();
auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context);
add_filter(additional_filters_info, "additional filter");
@ -1056,6 +1056,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto right_plan = std::move(right_join_tree_query_plan.query_plan);
auto right_plan_output_columns = right_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
// {
// WriteBufferFromOwnString buf;
// left_plan.explainPlan(buf, {.header = true, .actions = true});
// std::cerr << "left plan \n "<< buf.str() << std::endl;
// }
// {
// WriteBufferFromOwnString buf;
// right_plan.explainPlan(buf, {.header = true, .actions = true});
// std::cerr << "right plan \n "<< buf.str() << std::endl;
// }
JoinClausesAndActions join_clauses_and_actions;
JoinKind join_kind = join_node.getKind();
JoinStrictness join_strictness = join_node.getStrictness();

View File

@ -20,6 +20,7 @@
#include <Analyzer/Utils.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/TableFunctionNode.h>
@ -113,41 +114,96 @@ String JoinClause::dump() const
namespace
{
std::optional<JoinTableSide> extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node,
const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
const NameSet & left_table_expression_columns_names,
const NameSet & right_table_expression_columns_names,
using TableExpressionSet = std::unordered_set<const IQueryTreeNode *>;
TableExpressionSet extractTableExpressionsSet(const QueryTreeNodePtr & node)
{
TableExpressionSet res;
for (const auto & expr : extractTableExpressions(node, true))
res.insert(expr.get());
return res;
}
std::optional<JoinTableSide> extractJoinTableSideFromExpression(//const ActionsDAG::Node * expression_root_node,
const IQueryTreeNode * expression_root_node,
//const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
const TableExpressionSet & left_table_expressions,
const TableExpressionSet & right_table_expressions,
const JoinNode & join_node)
{
std::optional<JoinTableSide> table_side;
std::vector<const ActionsDAG::Node *> nodes_to_process;
std::vector<const IQueryTreeNode *> nodes_to_process;
nodes_to_process.push_back(expression_root_node);
// std::cerr << "==== extractJoinTableSideFromExpression\n";
// std::cerr << "inp nodes" << std::endl;
// for (const auto * node : join_expression_dag_input_nodes)
// std::cerr << reinterpret_cast<const void *>(node) << ' ' << node->result_name << std::endl;
// std::cerr << "l names" << std::endl;
// for (const auto & l : left_table_expression_columns_names)
// std::cerr << l << std::endl;
// std::cerr << "r names" << std::endl;
// for (const auto & r : right_table_expression_columns_names)
// std::cerr << r << std::endl;
// const auto * left_table_expr = join_node.getLeftTableExpression().get();
// const auto * right_table_expr = join_node.getRightTableExpression().get();
while (!nodes_to_process.empty())
{
const auto * node_to_process = nodes_to_process.back();
nodes_to_process.pop_back();
for (const auto & child : node_to_process->children)
nodes_to_process.push_back(child);
//std::cerr << "... " << reinterpret_cast<const void *>(node_to_process) << ' ' << node_to_process->result_name << std::endl;
if (!join_expression_dag_input_nodes.contains(node_to_process))
if (const auto * function_node = node_to_process->as<FunctionNode>())
{
for (const auto & child : function_node->getArguments())
nodes_to_process.push_back(child.get());
continue;
}
const auto * column_node = node_to_process->as<ColumnNode>();
if (!column_node)
continue;
const auto & input_name = node_to_process->result_name;
// if (!join_expression_dag_input_nodes.contains(node_to_process))
// continue;
bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name);
bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name);
const auto & input_name = column_node->getColumnName();
if (!left_table_expression_contains_input && !right_table_expression_contains_input)
// bool left_table_expression_contains_input = left_table_expression_columns_names.contains(input_name);
// bool right_table_expression_contains_input = right_table_expression_columns_names.contains(input_name);
// if (!left_table_expression_contains_input && !right_table_expression_contains_input)
// throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
// "JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns",
// join_node.formatASTForErrorMessage(),
// input_name,
// boost::join(left_table_expression_columns_names, ", "),
// boost::join(right_table_expression_columns_names, ", "));
const auto * column_source = column_node->getColumnSource().get();
if (!column_source)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No source for column {} in JOIN {}", input_name, join_node.formatASTForErrorMessage());
bool is_column_from_left_expr = left_table_expressions.contains(column_source);
bool is_column_from_right_expr = right_table_expressions.contains(column_source);
if (!is_column_from_left_expr && !is_column_from_right_expr)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} actions has column {} that do not exist in left {} or right {} table expression columns",
join_node.formatASTForErrorMessage(),
input_name,
boost::join(left_table_expression_columns_names, ", "),
boost::join(right_table_expression_columns_names, ", "));
column_source->formatASTForErrorMessage(),
join_node.getLeftTableExpression()->formatASTForErrorMessage(),
join_node.getRightTableExpression()->formatASTForErrorMessage());
auto input_table_side = left_table_expression_contains_input ? JoinTableSide::Left : JoinTableSide::Right;
auto input_table_side = is_column_from_left_expr ? JoinTableSide::Left : JoinTableSide::Right;
if (table_side && (*table_side) != input_table_side)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression contains column from left and right table",
@ -159,29 +215,58 @@ std::optional<JoinTableSide> extractJoinTableSideFromExpression(const ActionsDAG
return table_side;
}
void buildJoinClause(ActionsDAGPtr join_expression_dag,
const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
const ActionsDAG::Node * join_expressions_actions_node,
const NameSet & left_table_expression_columns_names,
const NameSet & right_table_expression_columns_names,
const ActionsDAG::Node * appendExpression(
ActionsDAGPtr & dag,
const QueryTreeNodePtr & expression,
const PlannerContextPtr & planner_context,
const JoinNode & join_node)
{
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions",
join_node.formatASTForErrorMessage());
return join_expression_dag_node_raw_pointers[0];
}
void buildJoinClause(
ActionsDAGPtr & left_dag,
ActionsDAGPtr & right_dag,
const PlannerContextPtr & planner_context,
//ActionsDAGPtr join_expression_dag,
//const std::unordered_set<const ActionsDAG::Node *> & join_expression_dag_input_nodes,
//const ActionsDAG::Node * join_expressions_actions_node,
const QueryTreeNodePtr & join_expression,
const TableExpressionSet & left_table_expressions,
const TableExpressionSet & right_table_expressions,
const JoinNode & join_node,
JoinClause & join_clause)
{
std::string function_name;
if (join_expressions_actions_node->function)
function_name = join_expressions_actions_node->function->getName();
//std::cerr << join_expression_dag->dumpDAG() << std::endl;
auto * function_node = join_expression->as<FunctionNode>();
if (function_node)
function_name = function_node->getFunction()->getName();
// if (join_expressions_actions_node->function)
// function_name = join_expressions_actions_node->function->getName();
/// For 'and' function go into children
if (function_name == "and")
{
for (const auto & child : join_expressions_actions_node->children)
for (const auto & child : function_node->getArguments())
{
buildJoinClause(join_expression_dag,
join_expression_dag_input_nodes,
buildJoinClause(//join_expression_dag,
//join_expression_dag_input_nodes,
left_dag,
right_dag,
planner_context,
child,
left_table_expression_columns_names,
right_table_expression_columns_names,
left_table_expressions,
right_table_expressions,
join_node,
join_clause);
}
@ -194,45 +279,49 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
if (function_name == "equals" || function_name == "isNotDistinctFrom" || is_asof_join_inequality)
{
const auto * left_child = join_expressions_actions_node->children.at(0);
const auto * right_child = join_expressions_actions_node->children.at(1);
const auto left_child = function_node->getArguments().getNodes().at(0);//join_expressions_actions_node->children.at(0);
const auto right_child = function_node->getArguments().getNodes().at(1); //join_expressions_actions_node->children.at(1);
auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child,
join_expression_dag_input_nodes,
left_table_expression_columns_names,
right_table_expression_columns_names,
auto left_expression_side_optional = extractJoinTableSideFromExpression(left_child.get(),
//join_expression_dag_input_nodes,
left_table_expressions,
right_table_expressions,
join_node);
auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child,
join_expression_dag_input_nodes,
left_table_expression_columns_names,
right_table_expression_columns_names,
auto right_expression_side_optional = extractJoinTableSideFromExpression(right_child.get(),
//join_expression_dag_input_nodes,
left_table_expressions,
right_table_expressions,
join_node);
if (!left_expression_side_optional && !right_expression_side_optional)
{
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} ON expression {} with constants is not supported",
join_node.formatASTForErrorMessage(),
join_expressions_actions_node->result_name);
"JOIN {} ON expression with constants is not supported",
join_node.formatASTForErrorMessage());
}
else if (left_expression_side_optional && !right_expression_side_optional)
{
join_clause.addCondition(*left_expression_side_optional, join_expressions_actions_node);
auto & dag = *left_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag;
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
join_clause.addCondition(*left_expression_side_optional, node);
}
else if (!left_expression_side_optional && right_expression_side_optional)
{
join_clause.addCondition(*right_expression_side_optional, join_expressions_actions_node);
auto & dag = *right_expression_side_optional == JoinTableSide::Left ? left_dag : right_dag;
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
join_clause.addCondition(*right_expression_side_optional, node);
}
else
{
// std::cerr << "===============\n";
auto left_expression_side = *left_expression_side_optional;
auto right_expression_side = *right_expression_side_optional;
if (left_expression_side != right_expression_side)
{
const ActionsDAG::Node * left_key = left_child;
const ActionsDAG::Node * right_key = right_child;
auto left_key = left_child;
auto right_key = right_child;
if (left_expression_side == JoinTableSide::Right)
{
@ -241,6 +330,9 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
asof_inequality = reverseASOFJoinInequality(asof_inequality);
}
const auto * left_node = appendExpression(left_dag, left_key, planner_context, join_node);
const auto * right_node = appendExpression(right_dag, right_key, planner_context, join_node);
if (is_asof_join_inequality)
{
if (join_clause.hasASOF())
@ -250,55 +342,66 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag,
join_node.formatASTForErrorMessage());
}
join_clause.addASOFKey(left_key, right_key, asof_inequality);
join_clause.addASOFKey(left_node, right_node, asof_inequality);
}
else
{
bool null_safe_comparison = function_name == "isNotDistinctFrom";
join_clause.addKey(left_key, right_key, null_safe_comparison);
join_clause.addKey(left_node, right_node, null_safe_comparison);
}
}
else
{
join_clause.addCondition(left_expression_side, join_expressions_actions_node);
auto & dag = left_expression_side == JoinTableSide::Left ? left_dag : right_dag;
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
join_clause.addCondition(left_expression_side, node);
}
}
return;
}
auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node,
join_expression_dag_input_nodes,
left_table_expression_columns_names,
right_table_expression_columns_names,
auto expression_side_optional = extractJoinTableSideFromExpression(//join_expressions_actions_node,
//join_expression_dag_input_nodes,
join_expression.get(),
left_table_expressions,
right_table_expressions,
join_node);
if (!expression_side_optional)
expression_side_optional = JoinTableSide::Right;
auto expression_side = *expression_side_optional;
join_clause.addCondition(expression_side, join_expressions_actions_node);
auto & dag = expression_side == JoinTableSide::Left ? left_dag : right_dag;
const auto * node = appendExpression(dag, join_expression, planner_context, join_node);
join_clause.addCondition(expression_side, node);
}
JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & join_expression_input_columns,
JoinClausesAndActions buildJoinClausesAndActions(//const ColumnsWithTypeAndName & join_expression_input_columns,
const ColumnsWithTypeAndName & left_table_expression_columns,
const ColumnsWithTypeAndName & right_table_expression_columns,
const JoinNode & join_node,
const PlannerContextPtr & planner_context)
{
ActionsDAGPtr join_expression_actions = std::make_shared<ActionsDAG>(join_expression_input_columns);
//ActionsDAGPtr join_expression_actions = std::make_shared<ActionsDAG>(join_expression_input_columns);
ActionsDAGPtr left_join_actions = std::make_shared<ActionsDAG>(left_table_expression_columns);
ActionsDAGPtr right_join_actions = std::make_shared<ActionsDAG>(right_table_expression_columns);
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", left_join_actions->dumpDAG());
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions cols {} ", right_join_actions->dumpDAG());
/** In ActionsDAG if input node has constant representation additional constant column is added.
* That way we cannot simply check that node has INPUT type during resolution of expression join table side.
* Put all nodes after actions dag initialization in set.
* To check if actions dag node is input column, we check if set contains it.
*/
const auto & join_expression_actions_nodes = join_expression_actions->getNodes();
// const auto & join_expression_actions_nodes = join_expression_actions->getNodes();
std::unordered_set<const ActionsDAG::Node *> join_expression_dag_input_nodes;
join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size());
for (const auto & node : join_expression_actions_nodes)
join_expression_dag_input_nodes.insert(&node);
// std::unordered_set<const ActionsDAG::Node *> join_expression_dag_input_nodes;
// join_expression_dag_input_nodes.reserve(join_expression_actions_nodes.size());
// for (const auto & node : join_expression_actions_nodes)
// join_expression_dag_input_nodes.insert(&node);
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
* If we do not ignore it, this function will be replaced by underlying constant.
@ -308,6 +411,9 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
* ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value);
*/
auto join_expression = join_node.getJoinExpression();
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->formatConvertedASTForErrorMessage());
// LOG_TRACE(&Poco::Logger::get("Planner"), "buildJoinClausesAndActions expr {} ", join_expression->dumpTree());
auto * constant_join_expression = join_expression->as<ConstantNode>();
if (constant_join_expression && constant_join_expression->hasSourceExpression())
@ -319,18 +425,18 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
"JOIN {} join expression expected function",
join_node.formatASTForErrorMessage());
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions",
join_node.formatASTForErrorMessage());
// PlannerActionsVisitor join_expression_visitor(planner_context);
// auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_expression);
// if (join_expression_dag_node_raw_pointers.size() != 1)
// throw Exception(ErrorCodes::LOGICAL_ERROR,
// "JOIN {} ON clause contains multiple expressions",
// join_node.formatASTForErrorMessage());
const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
if (!join_expressions_actions_root_node->function)
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"JOIN {} join expression expected function",
join_node.formatASTForErrorMessage());
// const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0];
// if (!join_expressions_actions_root_node->function)
// throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
// "JOIN {} join expression expected function",
// join_node.formatASTForErrorMessage());
size_t left_table_expression_columns_size = left_table_expression_columns.size();
@ -360,21 +466,27 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
join_right_actions_names_set.insert(right_table_expression_column.name);
}
JoinClausesAndActions result;
result.join_expression_actions = join_expression_actions;
auto join_left_table_expressions = extractTableExpressionsSet(join_node.getLeftTableExpression());
auto join_right_table_expressions = extractTableExpressionsSet(join_node.getRightTableExpression());
const auto & function_name = join_expressions_actions_root_node->function->getName();
JoinClausesAndActions result;
//result.join_expression_actions = join_expression_actions;
const auto & function_name = function_node->getFunction()->getName();
if (function_name == "or")
{
for (const auto & child : join_expressions_actions_root_node->children)
for (const auto & child : function_node->getArguments())
{
result.join_clauses.emplace_back();
buildJoinClause(join_expression_actions,
join_expression_dag_input_nodes,
buildJoinClause(//join_expression_actions,
//join_expression_dag_input_nodes,
left_join_actions,
right_join_actions,
planner_context,
child,
join_left_actions_names_set,
join_right_actions_names_set,
join_left_table_expressions,
join_right_table_expressions,
join_node,
result.join_clauses.back());
}
@ -383,11 +495,15 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
{
result.join_clauses.emplace_back();
buildJoinClause(join_expression_actions,
join_expression_dag_input_nodes,
join_expressions_actions_root_node,
join_left_actions_names_set,
join_right_actions_names_set,
buildJoinClause(
left_join_actions,
right_join_actions,
planner_context,
//join_expression_actions,
//join_expression_dag_input_nodes,
join_expression, //join_expressions_actions_root_node,
join_left_table_expressions,
join_right_table_expressions,
join_node,
result.join_clauses.back());
}
@ -412,12 +528,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (left_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &join_expression_actions->addFunction(and_function, left_filter_condition_nodes, {});
dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {});
else
dag_filter_condition_node = left_filter_condition_nodes[0];
join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node};
join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name);
}
@ -428,12 +544,12 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (right_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &join_expression_actions->addFunction(and_function, right_filter_condition_nodes, {});
dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {});
else
dag_filter_condition_node = right_filter_condition_nodes[0];
join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node};
join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name);
}
@ -470,10 +586,10 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
}
if (!left_key_node->result_type->equals(*common_type))
left_key_node = &join_expression_actions->addCast(*left_key_node, common_type, {});
left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {});
if (!right_key_node->result_type->equals(*common_type))
right_key_node = &join_expression_actions->addCast(*right_key_node, common_type, {});
right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {});
}
if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable())
@ -490,22 +606,29 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName &
* SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b)
*/
auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext());
left_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {});
right_key_node = &join_expression_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {});
left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {});
right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {});
}
join_expression_actions->addOrReplaceInOutputs(*left_key_node);
join_expression_actions->addOrReplaceInOutputs(*right_key_node);
left_join_actions->addOrReplaceInOutputs(*left_key_node);
right_join_actions->addOrReplaceInOutputs(*right_key_node);
add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name);
add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name);
}
}
result.left_join_expressions_actions = join_expression_actions->clone();
result.left_join_expressions_actions = left_join_actions->clone();
result.left_join_tmp_expression_actions = std::move(left_join_actions);
result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names);
result.right_join_expressions_actions = join_expression_actions->clone();
// for (const auto & name : join_right_actions_names)
// std::cerr << ".. " << name << std::endl;
// std::cerr << right_join_actions->dumpDAG() << std::endl;
result.right_join_expressions_actions = right_join_actions->clone();
result.right_join_tmp_expression_actions = std::move(right_join_actions);
result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names);
return result;
@ -525,10 +648,10 @@ JoinClausesAndActions buildJoinClausesAndActions(
"JOIN {} join does not have ON section",
join_node_typed.formatASTForErrorMessage());
auto join_expression_input_columns = left_table_expression_columns;
join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end());
// auto join_expression_input_columns = left_table_expression_columns;
// join_expression_input_columns.insert(join_expression_input_columns.end(), right_table_expression_columns.begin(), right_table_expression_columns.end());
return buildJoinClausesAndActions(join_expression_input_columns, left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context);
return buildJoinClausesAndActions(/*join_expression_input_columns,*/ left_table_expression_columns, right_table_expression_columns, join_node_typed, planner_context);
}
std::optional<bool> tryExtractConstantFromJoinNode(const QueryTreeNodePtr & join_node)

View File

@ -165,7 +165,8 @@ struct JoinClausesAndActions
/// Join clauses. Actions dag nodes point into join_expression_actions.
JoinClauses join_clauses;
/// Whole JOIN ON section expressions
ActionsDAGPtr join_expression_actions;
ActionsDAGPtr left_join_tmp_expression_actions;
ActionsDAGPtr right_join_tmp_expression_actions;
/// Left join expressions actions
ActionsDAGPtr left_join_expressions_actions;
/// Right join expressions actions

View File

@ -357,6 +357,7 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con
QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr & query_node,
const ContextPtr & context,
//PlannerContext & planner_context,
ResultReplacementMap * result_replacement_map)
{
auto & query_node_typed = query_node->as<QueryNode &>();
@ -406,6 +407,13 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr &
if (result_replacement_map)
result_replacement_map->emplace(table_expression, dummy_table_node);
dummy_table_node->setAlias(table_expression->getAlias());
// auto & src_table_expression_data = planner_context.getOrCreateTableExpressionData(table_expression);
// auto & dst_table_expression_data = planner_context.getOrCreateTableExpressionData(dummy_table_node);
// dst_table_expression_data = src_table_expression_data;
replacement_map.emplace(table_expression.get(), std::move(dummy_table_node));
}

View File

@ -436,7 +436,6 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
AggregateProjectionCandidates candidates;
const auto & parts = reading.getParts();
const auto & query_info = reading.getQueryInfo();
const auto metadata = reading.getStorageMetadata();
ContextPtr context = reading.getContext();
@ -481,8 +480,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock(
metadata,
candidate.dag->getRequiredColumnsNames(),
dag.filter_node != nullptr,
query_info,
(dag.filter_node ? dag.dag : nullptr),
parts,
max_added_blocks.get(),
context);

View File

@ -23,6 +23,8 @@
#include <Processors/Transforms/ReverseTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeIndexAnnoy.h>
#include <Storages/MergeTree/MergeTreeIndexUSearch.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreePrefetchedReadPool.h>
#include <Storages/MergeTree/MergeTreeReadPoolInOrder.h>
@ -1331,26 +1333,12 @@ static void buildIndexes(
const Names & primary_key_column_names = primary_key.column_names;
const auto & settings = context->getSettingsRef();
if (settings.query_plan_optimize_primary_key)
{
NameSet array_join_name_set;
if (query_info.syntax_analyzer_result)
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
indexes.emplace(ReadFromMergeTree::Indexes{{
filter_actions_dag,
context,
primary_key_column_names,
primary_key.expression}, {}, {}, {}, {}, false, {}});
}
else
{
indexes.emplace(ReadFromMergeTree::Indexes{{
query_info,
context,
primary_key_column_names,
primary_key.expression}, {}, {}, {}, {}, false, {}});
}
indexes.emplace(ReadFromMergeTree::Indexes{{
filter_actions_dag,
context,
primary_key_column_names,
primary_key.expression}, {}, {}, {}, {}, false, {}});
if (metadata_snapshot->hasPartitionKey())
{
@ -1363,11 +1351,7 @@ static void buildIndexes(
}
/// TODO Support row_policy_filter and additional_filters
if (settings.allow_experimental_analyzer)
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context);
else
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, query_info.query, context);
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(data, parts, filter_actions_dag, context);
MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context);
indexes->use_skip_indexes = settings.use_skip_indexes;
@ -1379,14 +1363,18 @@ static void buildIndexes(
if (!indexes->use_skip_indexes)
return;
const SelectQueryInfo * info = &query_info;
std::optional<SelectQueryInfo> info_copy;
if (settings.allow_experimental_analyzer)
auto get_query_info = [&]() -> const SelectQueryInfo &
{
info_copy.emplace(query_info);
info_copy->filter_actions_dag = filter_actions_dag;
info = &*info_copy;
}
if (settings.allow_experimental_analyzer)
{
info_copy.emplace(query_info);
info_copy->filter_actions_dag = filter_actions_dag;
return *info_copy;
}
return query_info;
};
std::unordered_set<std::string> ignored_index_names;
@ -1427,14 +1415,30 @@ static void buildIndexes(
if (inserted)
{
skip_indexes.merged_indices.emplace_back();
skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot);
skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(get_query_info(), metadata_snapshot);
}
skip_indexes.merged_indices[it->second].addIndex(index_helper);
}
else
{
auto condition = index_helper->createIndexCondition(*info, context);
MergeTreeIndexConditionPtr condition;
if (index_helper->isVectorSearch())
{
#ifdef ENABLE_ANNOY
if (const auto * annoy = typeid_cast<const MergeTreeIndexAnnoy *>(index_helper.get()))
condition = annoy->createIndexCondition(get_query_info(), context);
#endif
#ifdef ENABLE_USEARCH
if (const auto * usearch = typeid_cast<const MergeTreeIndexUSearch *>(index_helper.get()))
condition = usearch->createIndexCondition(get_query_info(), context);
#endif
if (!condition)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name);
}
else
condition = index_helper->createIndexCondition(filter_actions_dag, context);
if (!condition->alwaysUnknownOrTrue())
skip_indexes.useful_indices.emplace_back(index_helper, condition);
}
@ -1467,34 +1471,15 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
Poco::Logger * log,
std::optional<Indexes> & indexes)
{
const auto & settings = context->getSettingsRef();
if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key)
{
auto updated_query_info_with_filter_dag = query_info;
updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info);
return selectRangesToReadImpl(
std::move(parts),
std::move(alter_conversions),
metadata_snapshot_base,
metadata_snapshot,
updated_query_info_with_filter_dag,
context,
num_streams,
max_block_numbers_to_read,
data,
real_column_names,
sample_factor_column_queried,
log,
indexes);
}
auto updated_query_info_with_filter_dag = query_info;
updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info);
return selectRangesToReadImpl(
std::move(parts),
std::move(alter_conversions),
metadata_snapshot_base,
metadata_snapshot,
query_info,
updated_query_info_with_filter_dag,
context,
num_streams,
max_block_numbers_to_read,

View File

@ -37,19 +37,9 @@ ReadFromStorageStep::ReadFromStorageStep(
void ReadFromStorageStep::applyFilters()
{
std::shared_ptr<const KeyCondition> key_condition;
if (!context->getSettingsRef().allow_experimental_analyzer)
{
for (const auto & processor : pipe.getProcessors())
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
source->setKeyCondition(query_info, context);
}
else
{
for (const auto & processor : pipe.getProcessors())
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
source->setKeyCondition(filter_nodes.nodes, context);
}
for (const auto & processor : pipe.getProcessors())
if (auto * source = dynamic_cast<SourceWithKeyCondition *>(processor.get()))
source->setKeyCondition(filter_nodes.nodes, context);
}
}

View File

@ -16,15 +16,6 @@ protected:
/// Represents pushed down filters in source
std::shared_ptr<const KeyCondition> key_condition;
void setKeyConditionImpl(const SelectQueryInfo & query_info, ContextPtr context, const Block & keys)
{
key_condition = std::make_shared<const KeyCondition>(
query_info,
context,
keys.getNames(),
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
}
void setKeyConditionImpl(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context, const Block & keys)
{
std::unordered_map<std::string, DB::ColumnWithTypeAndName> node_name_to_input_column;
@ -46,10 +37,7 @@ public:
/// Set key_condition directly. It is used for filter push down in source.
virtual void setKeyCondition(const std::shared_ptr<const KeyCondition> & key_condition_) { key_condition = key_condition_; }
/// Set key_condition created by query_info and context. It is used for filter push down when allow_experimental_analyzer is false.
virtual void setKeyCondition(const SelectQueryInfo & /*query_info*/, ContextPtr /*context*/) { }
/// Set key_condition created by nodes and context. It is used for filter push down when allow_experimental_analyzer is true.
/// Set key_condition created by nodes and context.
virtual void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & /*nodes*/, ContextPtr /*context*/) { }
};
}

View File

@ -29,10 +29,14 @@
#include <Parsers/ASTLiteral.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/ISource.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <Processors/Sources/NullSource.h>
#include <Storages/AlterCommands.h>
#include <Storages/HDFS/ReadBufferFromHDFS.h>
#include <Storages/HDFS/AsynchronousReadBufferFromHDFS.h>
@ -123,7 +127,6 @@ public:
String compression_method_,
Block sample_block_,
ContextPtr context_,
const SelectQueryInfo & query_info_,
UInt64 max_block_size_,
const StorageHive & storage_,
const Names & text_input_field_names_ = {})
@ -140,7 +143,6 @@ public:
, text_input_field_names(text_input_field_names_)
, format_settings(getFormatSettings(getContext()))
, read_settings(getContext()->getReadSettings())
, query_info(query_info_)
{
to_read_block = sample_block;
@ -395,7 +397,6 @@ private:
const Names & text_input_field_names;
FormatSettings format_settings;
ReadSettings read_settings;
SelectQueryInfo query_info;
HiveFilePtr current_file;
String current_path;
@ -574,7 +575,7 @@ static HiveFilePtr createHiveFile(
HiveFiles StorageHive::collectHiveFilesFromPartition(
const Apache::Hadoop::Hive::Partition & partition,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs,
const ContextPtr & context_,
@ -638,7 +639,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition(
for (size_t i = 0; i < partition_names.size(); ++i)
ranges.emplace_back(fields[i]);
const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr);
const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr);
if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true)
return {};
}
@ -648,7 +649,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition(
hive_files.reserve(file_infos.size());
for (const auto & file_info : file_infos)
{
auto hive_file = getHiveFileIfNeeded(file_info, fields, query_info, hive_table_metadata, context_, prune_level);
auto hive_file = getHiveFileIfNeeded(file_info, fields, filter_actions_dag, hive_table_metadata, context_, prune_level);
if (hive_file)
{
LOG_TRACE(
@ -672,7 +673,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv
HiveFilePtr StorageHive::getHiveFileIfNeeded(
const FileInfo & file_info,
const FieldVector & fields,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const ContextPtr & context_,
PruneLevel prune_level) const
@ -706,7 +707,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded(
if (prune_level >= PruneLevel::File)
{
const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr);
const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr);
if (hive_file->useFileMinMaxIndex())
{
/// Load file level minmax index and apply
@ -758,10 +759,77 @@ bool StorageHive::supportsSubsetOfColumns() const
return format_name == "Parquet" || format_name == "ORC";
}
Pipe StorageHive::read(
class ReadFromHive : public SourceStepWithFilter
{
public:
std::string getName() const override { return "ReadFromHive"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
void applyFilters() override;
ReadFromHive(
Block header,
std::shared_ptr<StorageHive> storage_,
std::shared_ptr<StorageHiveSource::SourcesInfo> sources_info_,
HDFSBuilderWrapper builder_,
HDFSFSPtr fs_,
HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata_,
Block sample_block_,
Poco::Logger * log_,
ContextPtr context_,
size_t max_block_size_,
size_t num_streams_)
: SourceStepWithFilter(DataStream{.header = std::move(header)})
, storage(std::move(storage_))
, sources_info(std::move(sources_info_))
, builder(std::move(builder_))
, fs(std::move(fs_))
, hive_table_metadata(std::move(hive_table_metadata_))
, sample_block(std::move(sample_block_))
, log(log_)
, context(std::move(context_))
, max_block_size(max_block_size_)
, num_streams(num_streams_)
{
}
private:
std::shared_ptr<StorageHive> storage;
std::shared_ptr<StorageHiveSource::SourcesInfo> sources_info;
HDFSBuilderWrapper builder;
HDFSFSPtr fs;
HiveMetastoreClient::HiveTableMetadataPtr hive_table_metadata;
Block sample_block;
Poco::Logger * log;
ContextPtr context;
size_t max_block_size;
size_t num_streams;
std::optional<HiveFiles> hive_files;
void createFiles(const ActionsDAGPtr & filter_actions_dag);
};
void ReadFromHive::applyFilters()
{
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
createFiles(filter_actions_dag);
}
void ReadFromHive::createFiles(const ActionsDAGPtr & filter_actions_dag)
{
if (hive_files)
return;
hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context);
LOG_INFO(log, "Collect {} hive files to read", hive_files->size());
}
void StorageHive::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
SelectQueryInfo &,
ContextPtr context_,
QueryProcessingStage::Enum /* processed_stage */,
size_t max_block_size,
@ -774,15 +842,7 @@ Pipe StorageHive::read(
auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url);
auto hive_table_metadata = hive_metastore_client->getTableMetadata(hive_database, hive_table);
/// Collect Hive files to read
HiveFiles hive_files = collectHiveFiles(num_streams, query_info, hive_table_metadata, fs, context_);
LOG_INFO(log, "Collect {} hive files to read", hive_files.size());
if (hive_files.empty())
return {};
auto sources_info = std::make_shared<StorageHiveSource::SourcesInfo>();
sources_info->hive_files = std::move(hive_files);
sources_info->database_name = hive_database;
sources_info->table_name = hive_table;
sources_info->hive_metastore_client = hive_metastore_client;
@ -822,6 +882,36 @@ Pipe StorageHive::read(
sources_info->need_file_column = true;
}
auto this_ptr = std::static_pointer_cast<StorageHive>(shared_from_this());
auto reading = std::make_unique<ReadFromHive>(
StorageHiveSource::getHeader(sample_block, sources_info),
std::move(this_ptr),
std::move(sources_info),
std::move(builder),
std::move(fs),
std::move(hive_table_metadata),
std::move(sample_block),
log,
context_,
max_block_size,
num_streams);
query_plan.addStep(std::move(reading));
}
void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
createFiles(nullptr);
if (hive_files->empty())
{
pipeline.init(Pipe(std::make_shared<NullSource>(getOutputStream().header)));
return;
}
sources_info->hive_files = std::move(*hive_files);
if (num_streams > sources_info->hive_files.size())
num_streams = sources_info->hive_files.size();
@ -830,22 +920,29 @@ Pipe StorageHive::read(
{
pipes.emplace_back(std::make_shared<StorageHiveSource>(
sources_info,
hdfs_namenode_url,
format_name,
compression_method,
storage->hdfs_namenode_url,
storage->format_name,
storage->compression_method,
sample_block,
context_,
query_info,
context,
max_block_size,
*this,
text_input_field_names));
*storage,
storage->text_input_field_names));
}
return Pipe::unitePipes(std::move(pipes));
auto pipe = Pipe::unitePipes(std::move(pipes));
if (pipe.empty())
pipe = Pipe(std::make_shared<NullSource>(getOutputStream().header));
for (const auto & processor : pipe.getProcessors())
processors.emplace_back(processor);
pipeline.init(std::move(pipe));
}
HiveFiles StorageHive::collectHiveFiles(
size_t max_threads,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs,
const ContextPtr & context_,
@ -871,7 +968,7 @@ HiveFiles StorageHive::collectHiveFiles(
[&]()
{
auto hive_files_in_partition
= collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level);
= collectHiveFilesFromPartition(partition, filter_actions_dag, hive_table_metadata, fs, context_, prune_level);
if (!hive_files_in_partition.empty())
{
std::lock_guard lock(hive_files_mutex);
@ -897,7 +994,7 @@ HiveFiles StorageHive::collectHiveFiles(
pool.scheduleOrThrowOnError(
[&]()
{
auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level);
auto hive_file = getHiveFileIfNeeded(file_info, {}, filter_actions_dag, hive_table_metadata, context_, prune_level);
if (hive_file)
{
std::lock_guard lock(hive_files_mutex);
@ -925,13 +1022,12 @@ NamesAndTypesList StorageHive::getVirtuals() const
std::optional<UInt64> StorageHive::totalRows(const Settings & settings) const
{
/// query_info is not used when prune_level == PruneLevel::None
SelectQueryInfo query_info;
return totalRowsImpl(settings, query_info, getContext(), PruneLevel::None);
return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None);
}
std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const
std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const
{
return totalRowsImpl(context_->getSettingsRef(), query_info, context_, PruneLevel::Partition);
return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition);
}
void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const
@ -946,7 +1042,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt
}
std::optional<UInt64>
StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const
StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const
{
/// Row-based format like Text doesn't support totalRowsByPartitionPredicate
if (!supportsSubsetOfColumns())
@ -958,7 +1054,7 @@ StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & qu
HDFSFSPtr fs = createHDFSFS(builder.get());
HiveFiles hive_files = collectHiveFiles(
settings.max_threads,
query_info,
filter_actions_dag,
hive_table_metadata,
fs,
context_,

View File

@ -42,10 +42,11 @@ public:
bool supportsSubcolumns() const override { return true; }
Pipe read(
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
SelectQueryInfo &,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
@ -58,9 +59,12 @@ public:
bool supportsSubsetOfColumns() const;
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override;
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override;
protected:
friend class ReadFromHive;
private:
using FileFormat = IHiveFile::FileFormat;
using FileInfo = HiveMetastoreClient::FileInfo;
@ -88,7 +92,7 @@ private:
HiveFiles collectHiveFiles(
size_t max_threads,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs,
const ContextPtr & context_,
@ -96,7 +100,7 @@ private:
HiveFiles collectHiveFilesFromPartition(
const Apache::Hadoop::Hive::Partition & partition,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs,
const ContextPtr & context_,
@ -105,7 +109,7 @@ private:
HiveFilePtr getHiveFileIfNeeded(
const FileInfo & file_info,
const FieldVector & fields,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata,
const ContextPtr & context_,
PruneLevel prune_level = PruneLevel::Max) const;
@ -113,7 +117,7 @@ private:
void lazyInitialize();
std::optional<UInt64>
totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const;
totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const;
String hive_metastore_url;

View File

@ -669,7 +669,7 @@ public:
virtual std::optional<UInt64> totalRows(const Settings &) const { return {}; }
/// Same as above but also take partition predicate into account.
virtual std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const { return {}; }
virtual std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; }
/// If it is possible to quickly determine exact number of bytes for the table on storage:
/// - memory (approximated, resident)

View File

@ -762,92 +762,6 @@ void KeyCondition::getAllSpaceFillingCurves()
}
}
KeyCondition::KeyCondition(
const ASTPtr & query,
const ASTs & additional_filter_asts,
Block block_with_constants,
PreparedSetsPtr prepared_sets,
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_,
NameSet array_joined_column_names_,
bool single_point_,
bool strict_)
: key_expr(key_expr_)
, key_subexpr_names(getAllSubexpressionNames(*key_expr))
, array_joined_column_names(std::move(array_joined_column_names_))
, single_point(single_point_)
, strict(strict_)
{
size_t key_index = 0;
for (const auto & name : key_column_names)
{
if (!key_columns.contains(name))
{
key_columns[name] = key_columns.size();
key_indices.push_back(key_index);
}
++key_index;
}
if (context->getSettingsRef().analyze_index_with_space_filling_curves)
getAllSpaceFillingCurves();
ASTPtr filter_node;
if (query)
filter_node = buildFilterNode(query, additional_filter_asts);
if (!filter_node)
{
has_filter = false;
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
has_filter = true;
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
* when parsing the AST into internal RPN representation.
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
* are pushed down and applied (when possible) to leaf nodes.
*/
auto inverted_filter_node = DB::cloneASTWithInversionPushDown(filter_node);
RPNBuilder<RPNElement> builder(
inverted_filter_node,
std::move(context),
std::move(block_with_constants),
std::move(prepared_sets),
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
findHyperrectanglesForArgumentsOfSpaceFillingCurves();
}
KeyCondition::KeyCondition(
const SelectQueryInfo & query_info,
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_,
bool single_point_,
bool strict_)
: KeyCondition(
query_info.query,
query_info.filter_asts,
KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context),
query_info.prepared_sets,
context,
key_column_names,
key_expr_,
query_info.syntax_analyzer_result ? query_info.syntax_analyzer_result->getArrayJoinSourceNameSet() : NameSet{},
single_point_,
strict_)
{
}
KeyCondition::KeyCondition(
ActionsDAGPtr filter_dag,
ContextPtr context,
@ -883,6 +797,13 @@ KeyCondition::KeyCondition(
has_filter = true;
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
* when parsing the AST into internal RPN representation.
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
* are pushed down and applied (when possible) to leaf nodes.
*/
auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context);
assert(inverted_dag->getOutputs().size() == 1);

View File

@ -39,30 +39,6 @@ struct ActionDAGNodes;
class KeyCondition
{
public:
/// Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters
KeyCondition(
const ASTPtr & query,
const ASTs & additional_filter_asts,
Block block_with_constants,
PreparedSetsPtr prepared_sets_,
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr,
NameSet array_joined_column_names,
bool single_point_ = false,
bool strict_ = false);
/** Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters.
* Select query, additional filters, prepared sets are initialized using query info.
*/
KeyCondition(
const SelectQueryInfo & query_info,
ContextPtr context,
const Names & key_column_names,
const ExpressionActionsPtr & key_expr_,
bool single_point_ = false,
bool strict_ = false);
/// Construct key condition from ActionsDAG nodes
KeyCondition(
ActionsDAGPtr filter_dag,

View File

@ -1075,26 +1075,30 @@ Block MergeTreeData::getBlockWithVirtualPartColumns(const MergeTreeData::DataPar
std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
const SelectQueryInfo & query_info, ContextPtr local_context, const DataPartsVector & parts) const
const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const
{
if (parts.empty())
return 0u;
auto metadata_snapshot = getInMemoryMetadataPtr();
ASTPtr expression_ast;
Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */);
// Generate valid expressions for filtering
bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast);
auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr);
PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */);
// Generate valid expressions for filtering
bool valid = true;
for (const auto * input : filter_dag->getInputs())
if (!virtual_columns_block.has(input->result_name))
valid = false;
PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */);
if (partition_pruner.isUseless() && !valid)
return {};
std::unordered_set<String> part_values;
if (valid && expression_ast)
if (valid)
{
virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */);
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, local_context, expression_ast);
VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context);
part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
if (part_values.empty())
return 0;
@ -6621,8 +6625,7 @@ using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
Block MergeTreeData::getMinMaxCountProjectionBlock(
const StorageMetadataPtr & metadata_snapshot,
const Names & required_columns,
bool has_filter,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_dag,
const DataPartsVector & parts,
const PartitionIdToMaxBlock * max_block_numbers_to_read,
ContextPtr query_context) const
@ -6672,7 +6675,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
Block virtual_columns_block;
auto virtual_block = getSampleBlockWithVirtualColumns();
bool has_virtual_column = std::any_of(required_columns.begin(), required_columns.end(), [&](const auto & name) { return virtual_block.has(name); });
if (has_virtual_column || has_filter)
if (has_virtual_column || filter_dag)
{
virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */);
if (virtual_columns_block.rows() == 0)
@ -6684,7 +6687,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
std::optional<PartitionPruner> partition_pruner;
std::optional<KeyCondition> minmax_idx_condition;
DataTypes minmax_columns_types;
if (has_filter)
if (filter_dag)
{
if (metadata_snapshot->hasPartitionKey())
{
@ -6693,16 +6696,15 @@ Block MergeTreeData::getMinMaxCountProjectionBlock(
minmax_columns_types = getMinMaxColumnsTypes(partition_key);
minmax_idx_condition.emplace(
query_info, query_context, minmax_columns_names,
filter_dag, query_context, minmax_columns_names,
getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context)));
partition_pruner.emplace(metadata_snapshot, query_info, query_context, false /* strict */);
partition_pruner.emplace(metadata_snapshot, filter_dag, query_context, false /* strict */);
}
const auto * predicate = filter_dag->getOutputs().at(0);
// Generate valid expressions for filtering
ASTPtr expression_ast;
VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, query_context, virtual_columns_block, expression_ast);
if (expression_ast)
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, query_context, expression_ast);
VirtualColumnUtils::filterBlockWithPredicate(predicate, virtual_columns_block, query_context);
rows = virtual_columns_block.rows();
part_name_column = virtual_columns_block.getByName("_part").column;

View File

@ -404,8 +404,7 @@ public:
Block getMinMaxCountProjectionBlock(
const StorageMetadataPtr & metadata_snapshot,
const Names & required_columns,
bool has_filter,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_dag,
const DataPartsVector & parts,
const PartitionIdToMaxBlock * max_block_numbers_to_read,
ContextPtr query_context) const;
@ -1222,7 +1221,7 @@ protected:
boost::iterator_range<DataPartIteratorByStateAndInfo> range, const ColumnsDescription & storage_columns);
std::optional<UInt64> totalRowsByPartitionPredicateImpl(
const SelectQueryInfo & query_info, ContextPtr context, const DataPartsVector & parts) const;
const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const;
static decltype(auto) getStateModifier(DataPartState state)
{

View File

@ -461,7 +461,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(
= {ColumnWithTypeAndName(part_offset_type->createColumn(), part_offset_type, "_part_offset"),
ColumnWithTypeAndName(part_type->createColumn(), part_type, "_part")};
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample);
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample);
if (!dag)
return;
@ -487,7 +487,7 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
if (!filter_dag)
return {};
auto sample = data.getSampleBlockWithVirtualColumns();
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), sample);
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_dag->getOutputs().at(0), &sample);
if (!dag)
return {};
@ -496,34 +496,6 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
}
std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const ASTPtr & query,
ContextPtr context)
{
std::unordered_set<String> part_values;
ASTPtr expression_ast;
auto virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, true /* one_part */);
if (virtual_columns_block.rows() == 0)
return {};
// Generate valid expressions for filtering
VirtualColumnUtils::prepareFilterBlockWithQuery(query, context, virtual_columns_block, expression_ast);
// If there is still something left, fill the virtual block and do the filtering.
if (expression_ast)
{
virtual_columns_block = data.getBlockWithVirtualPartColumns(parts, false /* one_part */);
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context, expression_ast);
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
}
return {};
}
void MergeTreeDataSelectExecutor::filterPartsByPartition(
const std::optional<PartitionPruner> & partition_pruner,
const std::optional<KeyCondition> & minmax_idx_condition,

View File

@ -168,12 +168,6 @@ public:
/// If possible, filter using expression on virtual columns.
/// Example: SELECT count() FROM table WHERE _part = 'part_name'
/// If expression found, return a set with allowed part names (std::nullopt otherwise).
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const ASTPtr & query,
ContextPtr context);
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int INCORRECT_NUMBER_OF_COLUMNS;
extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
template <typename Distance>
@ -331,6 +332,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec
return std::make_shared<MergeTreeIndexConditionAnnoy>(index, query, distance_function, context);
};
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
}
MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index)
{
static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2;

View File

@ -88,7 +88,7 @@ private:
};
class MergeTreeIndexAnnoy : public IMergeTreeIndex
class MergeTreeIndexAnnoy final : public IMergeTreeIndex
{
public:
@ -98,7 +98,9 @@ public:
MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override;
bool isVectorSearch() const override { return true; }
private:
const UInt64 trees;

View File

@ -43,9 +43,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
}
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
{
return std::make_shared<MergeTreeIndexConditionBloomFilter>(query_info, context, index.sample_block, hash_functions);
return std::make_shared<MergeTreeIndexConditionBloomFilter>(filter_actions_dag, context, index.sample_block, hash_functions);
}
static void assertIndexColumnsType(const Block & header)

View File

@ -20,7 +20,7 @@ public:
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override;
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
private:
size_t bits_per_row;

View File

@ -97,39 +97,18 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
}
MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter(
const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_)
: WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_)
const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_)
: WithContext(context_), header(header_), hash_functions(hash_functions_)
{
if (context_->getSettingsRef().allow_experimental_analyzer)
{
if (!query_info.filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
RPNBuilder<RPNElement> builder(
query_info.filter_actions_dag->getOutputs().at(0),
context_,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
return;
}
ASTPtr filter_node = buildFilterNode(query_info.query);
if (!filter_node)
if (!filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
RPNBuilder<RPNElement> builder(
filter_node,
filter_actions_dag->getOutputs().at(0),
context_,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
}

View File

@ -44,7 +44,7 @@ public:
std::vector<std::pair<size_t, ColumnPtr>> predicate;
};
MergeTreeIndexConditionBloomFilter(const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_);
MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_);
bool alwaysUnknownOrTrue() const override;
@ -58,7 +58,6 @@ public:
private:
const Block & header;
const SelectQueryInfo & query_info;
const size_t hash_functions;
std::vector<RPNElement> rpn;

View File

@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
}
MergeTreeConditionFullText::MergeTreeConditionFullText(
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
ContextPtr context,
const Block & index_sample_block,
const BloomFilterParameters & params_,
@ -147,38 +147,16 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
, index_data_types(index_sample_block.getNamesAndTypesList().getTypes())
, params(params_)
, token_extractor(token_extactor_)
, prepared_sets(query_info.prepared_sets)
{
if (context->getSettingsRef().allow_experimental_analyzer)
{
if (!query_info.filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
RPNBuilder<RPNElement> builder(
query_info.filter_actions_dag->getOutputs().at(0),
context,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
return;
}
ASTPtr filter_node = buildFilterNode(query_info.query);
if (!filter_node)
if (!filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
RPNBuilder<RPNElement> builder(
filter_node,
filter_actions_dag->getOutputs().at(0),
context,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
}
@ -747,9 +725,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const
}
MergeTreeIndexConditionPtr MergeTreeIndexFullText::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
const ActionsDAGPtr & filter_dag, ContextPtr context) const
{
return std::make_shared<MergeTreeConditionFullText>(query, context, index.sample_block, params, token_extractor.get());
return std::make_shared<MergeTreeConditionFullText>(filter_dag, context, index.sample_block, params, token_extractor.get());
}
MergeTreeIndexPtr bloomFilterIndexCreator(

View File

@ -62,7 +62,7 @@ class MergeTreeConditionFullText final : public IMergeTreeIndexCondition
{
public:
MergeTreeConditionFullText(
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
ContextPtr context,
const Block & index_sample_block,
const BloomFilterParameters & params_,
@ -144,9 +144,6 @@ private:
BloomFilterParameters params;
TokenExtractorPtr token_extractor;
RPN rpn;
/// Sets from syntax analyzer.
PreparedSetsPtr prepared_sets;
};
class MergeTreeIndexFullText final : public IMergeTreeIndex
@ -166,7 +163,7 @@ public:
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override;
const ActionsDAGPtr & filter_dag, ContextPtr context) const override;
BloomFilterParameters params;
/// Function for selecting next token.

View File

@ -79,7 +79,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(cons
}
MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition(
const SelectQueryInfo &, ContextPtr) const
const ActionsDAGPtr &, ContextPtr) const
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported");
}

View File

@ -70,7 +70,7 @@ public:
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override;
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override;

View File

@ -184,7 +184,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos,
}
MergeTreeConditionInverted::MergeTreeConditionInverted(
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
ContextPtr context_,
const Block & index_sample_block,
const GinFilterParameters & params_,
@ -192,41 +192,20 @@ MergeTreeConditionInverted::MergeTreeConditionInverted(
: WithContext(context_), header(index_sample_block)
, params(params_)
, token_extractor(token_extactor_)
, prepared_sets(query_info.prepared_sets)
{
if (context_->getSettingsRef().allow_experimental_analyzer)
{
if (!query_info.filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
rpn = std::move(
RPNBuilder<RPNElement>(
query_info.filter_actions_dag->getOutputs().at(0), context_,
[&](const RPNBuilderTreeNode & node, RPNElement & out)
{
return this->traverseAtomAST(node, out);
}).extractRPN());
return;
}
ASTPtr filter_node = buildFilterNode(query_info.query);
if (!filter_node)
if (!filter_actions_dag)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
RPNBuilder<RPNElement> builder(
filter_node,
context_,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); });
rpn = std::move(builder).extractRPN();
rpn = std::move(
RPNBuilder<RPNElement>(
filter_actions_dag->getOutputs().at(0), context_,
[&](const RPNBuilderTreeNode & node, RPNElement & out)
{
return this->traverseAtomAST(node, out);
}).extractRPN());
}
/// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue
@ -721,9 +700,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart
}
MergeTreeIndexConditionPtr MergeTreeIndexInverted::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
{
return std::make_shared<MergeTreeConditionInverted>(query, context, index.sample_block, params, token_extractor.get());
return std::make_shared<MergeTreeConditionInverted>(filter_actions_dag, context, index.sample_block, params, token_extractor.get());
};
MergeTreeIndexPtr invertedIndexCreator(

View File

@ -64,7 +64,7 @@ class MergeTreeConditionInverted final : public IMergeTreeIndexCondition, WithCo
{
public:
MergeTreeConditionInverted(
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
ContextPtr context,
const Block & index_sample_block,
const GinFilterParameters & params_,
@ -169,7 +169,7 @@ public:
MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
GinFilterParameters params;
/// Function for selecting next token.

View File

@ -156,20 +156,17 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s
namespace
{
KeyCondition buildCondition(const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
KeyCondition buildCondition(const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context)
{
if (context->getSettingsRef().allow_experimental_analyzer)
return KeyCondition{query_info.filter_actions_dag, context, index.column_names, index.expression};
return KeyCondition{query_info, context, index.column_names, index.expression};
return KeyCondition{filter_actions_dag, context, index.column_names, index.expression};
}
}
MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax(
const IndexDescription & index, const SelectQueryInfo & query_info, ContextPtr context)
const IndexDescription & index, const ActionsDAGPtr & filter_actions_dag, ContextPtr context)
: index_data_types(index.data_types)
, condition(buildCondition(index, query_info, context))
, condition(buildCondition(index, filter_actions_dag, context))
{
}
@ -200,9 +197,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const Me
}
MergeTreeIndexConditionPtr MergeTreeIndexMinMax::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
{
return std::make_shared<MergeTreeIndexConditionMinMax>(index, query, context);
return std::make_shared<MergeTreeIndexConditionMinMax>(index, filter_actions_dag, context);
}
MergeTreeIndexFormat MergeTreeIndexMinMax::getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & relative_path_prefix) const

View File

@ -52,7 +52,7 @@ class MergeTreeIndexConditionMinMax final : public IMergeTreeIndexCondition
public:
MergeTreeIndexConditionMinMax(
const IndexDescription & index,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_actions_dag,
ContextPtr context);
bool alwaysUnknownOrTrue() const override;
@ -79,7 +79,7 @@ public:
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override;
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
const char* getSerializedFileExtension() const override { return ".idx2"; }
MergeTreeIndexFormat getDeserializedFormat(const IDataPartStorage & data_part_storage, const std::string & path_prefix) const override; /// NOLINT

View File

@ -247,7 +247,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
const String & index_name_,
const Block & index_sample_block,
size_t max_rows_,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_dag,
ContextPtr context)
: index_name(index_name_)
, max_rows(max_rows_)
@ -256,42 +256,20 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
if (!key_columns.contains(name))
key_columns.insert(name);
if (context->getSettingsRef().allow_experimental_analyzer)
{
if (!query_info.filter_actions_dag)
return;
if (!filter_dag)
return;
if (checkDAGUseless(*query_info.filter_actions_dag->getOutputs().at(0), context))
return;
if (checkDAGUseless(*filter_dag->getOutputs().at(0), context))
return;
const auto * filter_node = query_info.filter_actions_dag->getOutputs().at(0);
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG({filter_node}, {}, context);
const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0);
auto filter_actions_dag = filter_dag->clone();
const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0);
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node);
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node);
filter_actions_dag->removeUnusedActions();
actions = std::make_shared<ExpressionActions>(filter_actions_dag);
}
else
{
ASTPtr ast_filter_node = buildFilterNode(query_info.query);
if (!ast_filter_node)
return;
if (checkASTUseless(ast_filter_node))
return;
auto expression_ast = ast_filter_node->clone();
/// Replace logical functions with bit functions.
/// Working with UInt8: last bit = can be true, previous = can be false (Like src/Storages/MergeTree/BoolMask.h).
traverseAST(expression_ast);
auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList());
actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true);
}
filter_actions_dag->removeUnusedActions();
actions = std::make_shared<ExpressionActions>(filter_actions_dag);
}
bool MergeTreeIndexConditionSet::alwaysUnknownOrTrue() const
@ -704,9 +682,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const Merge
}
MergeTreeIndexConditionPtr MergeTreeIndexSet::createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
{
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, query, context);
return std::make_shared<MergeTreeIndexConditionSet>(index.name, index.sample_block, max_rows, filter_actions_dag, context);
}
MergeTreeIndexPtr setIndexCreator(const IndexDescription & index)

View File

@ -87,7 +87,7 @@ public:
const String & index_name_,
const Block & index_sample_block,
size_t max_rows_,
const SelectQueryInfo & query_info,
const ActionsDAGPtr & filter_dag,
ContextPtr context);
bool alwaysUnknownOrTrue() const override;
@ -149,7 +149,7 @@ public:
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override;
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
size_t max_rows = 0;
};

View File

@ -36,6 +36,7 @@ namespace ErrorCodes
extern const int INCORRECT_NUMBER_OF_COLUMNS;
extern const int INCORRECT_QUERY;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
namespace
@ -366,6 +367,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel
return std::make_shared<MergeTreeIndexConditionUSearch>(index, query, distance_function, context);
};
MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
}
MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index)
{
static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2;

View File

@ -100,7 +100,9 @@ public:
MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override;
bool isVectorSearch() const override { return true; }
private:
const String distance_function;

View File

@ -170,7 +170,9 @@ struct IMergeTreeIndex
}
virtual MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query_info, ContextPtr context) const = 0;
const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const = 0;
virtual bool isVectorSearch() const { return false; }
virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondition(
const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const

View File

@ -9,10 +9,7 @@ namespace
KeyCondition buildKeyCondition(const KeyDescription & partition_key, const SelectQueryInfo & query_info, ContextPtr context, bool strict)
{
if (context->getSettingsRef().allow_experimental_analyzer)
return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
return {query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
return {query_info.filter_actions_dag, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict};
}
}

View File

@ -202,17 +202,6 @@ public:
traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context));
}
RPNBuilder(const ASTPtr & filter_node,
ContextPtr query_context_,
Block block_with_constants_,
PreparedSetsPtr prepared_sets_,
const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_)
: tree_context(std::move(query_context_), std::move(block_with_constants_), std::move(prepared_sets_))
, extract_atom_from_tree_function(extract_atom_from_tree_function_)
{
traverseTree(RPNBuilderTreeNode(filter_node.get(), tree_context));
}
RPNElements && extractRPN() && { return std::move(rpn_elements); }
private:

View File

@ -1056,11 +1056,6 @@ StorageFileSource::~StorageFileSource()
beforeDestroy();
}
void StorageFileSource::setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_)
{
setKeyConditionImpl(query_info_, context_, block_for_format);
}
void StorageFileSource::setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_)
{
setKeyConditionImpl(nodes, context_, block_for_format);

View File

@ -256,8 +256,6 @@ private:
return storage->getName();
}
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override;
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override;
bool tryGetCountFromCache(const struct stat & file_stat);

View File

@ -261,10 +261,10 @@ std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const
return getTotalActiveSizeInRows();
}
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const
{
auto parts = getVisibleDataPartsVector(local_context);
return totalRowsByPartitionPredicateImpl(query_info, local_context, parts);
return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts);
}
std::optional<UInt64> StorageMergeTree::totalBytes(const Settings &) const

View File

@ -66,7 +66,7 @@ public:
size_t num_streams) override;
std::optional<UInt64> totalRows(const Settings &) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override;
std::optional<UInt64> totalBytes(const Settings &) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings &) const override;

View File

@ -5470,11 +5470,11 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalRows(const Settings & set
return res;
}
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr local_context) const
std::optional<UInt64> StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const
{
DataPartsVector parts;
foreachActiveParts([&](auto & part) { parts.push_back(part); }, local_context->getSettingsRef().select_sequential_consistency);
return totalRowsByPartitionPredicateImpl(query_info, local_context, parts);
return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts);
}
std::optional<UInt64> StorageReplicatedMergeTree::totalBytes(const Settings & settings) const

View File

@ -163,7 +163,7 @@ public:
size_t num_streams) override;
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings & settings) const override;

View File

@ -151,11 +151,6 @@ public:
String getName() const override;
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override
{
setKeyConditionImpl(query_info_, context_, sample_block);
}
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override
{
setKeyConditionImpl(nodes, context_, sample_block);

View File

@ -172,11 +172,6 @@ public:
String getName() const override { return name; }
void setKeyCondition(const SelectQueryInfo & query_info_, ContextPtr context_) override
{
setKeyConditionImpl(query_info_, context_, block_for_format);
}
void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override
{
setKeyConditionImpl(nodes, context_, block_for_format);

View File

@ -104,7 +104,7 @@ ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr
MutableColumnPtr database_column = ColumnString::create();
MutableColumnPtr engine_column;
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, sample);
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample);
if (dag)
{
bool filter_by_engine = false;

View File

@ -253,19 +253,7 @@ static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & co
if (!future_set->get())
{
if (auto * set_from_subquery = typeid_cast<FutureSetFromSubquery *>(future_set.get()))
{
auto plan = set_from_subquery->build(context);
if (!plan)
continue;
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
pipeline.complete(std::make_shared<EmptySink>(Block()));
CompletedPipelineExecutor executor(pipeline);
executor.execute();
}
set_from_subquery->buildSetInplace(context);
}
}
}
@ -406,7 +394,7 @@ ActionsDAGPtr createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, con
}
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
return splitFilterDagForAllowedInputs(predicate, block);
return splitFilterDagForAllowedInputs(predicate, &block);
}
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ActionsDAGPtr & dag, const NamesAndTypesList & virtual_columns, const ContextPtr & context)
@ -480,7 +468,7 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo
static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
const ActionsDAG::Node * node,
const Block & allowed_inputs,
const Block * allowed_inputs,
ActionsDAG::Nodes & additional_nodes)
{
if (node->type == ActionsDAG::ActionType::FUNCTION)
@ -555,13 +543,13 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs(
}
}
if (!canEvaluateSubtree(node, allowed_inputs))
if (allowed_inputs && !canEvaluateSubtree(node, *allowed_inputs))
return nullptr;
return node;
}
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs)
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs)
{
if (!predicate)
return nullptr;
@ -576,7 +564,7 @@ ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate,
void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, ContextPtr context)
{
auto dag = splitFilterDagForAllowedInputs(predicate, block);
auto dag = splitFilterDagForAllowedInputs(predicate, &block);
if (dag)
filterBlockWithDAG(dag, block, context);
}

View File

@ -42,7 +42,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block,
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context);
/// Extract a part of predicate that can be evaluated using only columns from input_names.
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block & allowed_inputs);
ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs);
/// Extract from the input stream a set of `name` column values
template <typename T>

View File

@ -1,6 +1,7 @@
#include <Storages/buildQueryTreeForShard.h>
#include <Analyzer/createUniqueTableAliases.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/IQueryTreeNode.h>
@ -372,6 +373,10 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN
removeGroupingFunctionSpecializations(query_tree_to_modify);
// std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl;
createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext());
// std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl;
return query_tree_to_modify;
}

View File

@ -72,10 +72,20 @@ class BuildConfig:
include_paths=[
"./src",
"./contrib/*-cmake",
"./contrib/consistent-hashing",
"./contrib/murmurhash",
"./contrib/libfarmhash",
"./contrib/pdqsort",
"./contrib/cityhash102",
"./contrib/sparse-checkout",
"./contrib/libmetrohash",
"./contrib/update-submodules.sh",
"./contrib/CMakeLists.txt",
"./cmake",
"./base",
"./programs",
"./packages",
"./docker/packager/packager",
],
exclude_files=[".md"],
docker=["clickhouse/binary-builder"],

View File

@ -10,7 +10,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.regression_for_in_operator_view
TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view
WHERE
FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -27,7 +27,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.regression_for_in_operator_view
TABLE id: 3, alias: __table1, table_name: default.regression_for_in_operator_view
WHERE
FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -34,7 +34,7 @@ QUERY id: 0
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -77,7 +77,7 @@ QUERY id: 0
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
k UInt64
s UInt64
@ -86,7 +86,7 @@ QUERY id: 0
COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7
COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.bug
TABLE id: 7, alias: __table2, table_name: default.bug
WHERE
FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -151,7 +151,7 @@ QUERY id: 0
COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3
CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
SETTINGS allow_experimental_analyzer=1
21 1
22 1
@ -184,7 +184,7 @@ QUERY id: 0
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
SETTINGS allow_experimental_analyzer=1
1 21
1 22
@ -222,7 +222,7 @@ QUERY id: 0
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -265,7 +265,7 @@ QUERY id: 0
COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
k UInt64
s UInt64
@ -274,7 +274,7 @@ QUERY id: 0
COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7
COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.bug
TABLE id: 7, alias: __table2, table_name: default.bug
WHERE
FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -347,7 +347,7 @@ QUERY id: 0
COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3
CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
SETTINGS allow_experimental_analyzer=1
21 1
22 1
@ -380,7 +380,7 @@ QUERY id: 0
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
SETTINGS allow_experimental_analyzer=1
21 1
22 1
@ -413,5 +413,5 @@ QUERY id: 0
COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3
CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8)
JOIN TREE
TABLE id: 3, table_name: default.bug
TABLE id: 3, alias: __table1, table_name: default.bug
SETTINGS allow_experimental_analyzer=1

View File

@ -49,7 +49,7 @@ QUERY id: 0
LIST id: 9, nodes: 1
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
JOIN TREE
TABLE_FUNCTION id: 11, table_function_name: numbers
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
@ -124,7 +124,7 @@ QUERY id: 0
LIST id: 9, nodes: 1
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
JOIN TREE
TABLE_FUNCTION id: 11, table_function_name: numbers
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
@ -194,7 +194,7 @@ QUERY id: 0
COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7
CONSTANT id: 11, constant_value: UInt64_5, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 7, table_function_name: numbers
TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32
@ -276,7 +276,7 @@ QUERY id: 0
LIST id: 9, nodes: 1
COLUMN id: 10, column_name: number, result_type: UInt64, source_id: 11
JOIN TREE
TABLE_FUNCTION id: 11, table_function_name: numbers
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10000000, constant_value_type: UInt32

View File

@ -49,14 +49,14 @@ QUERY id: 0
LIST id: 3, nodes: 1
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
JOIN TREE
QUERY id: 5, is_subquery: 1
QUERY id: 5, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 6, nodes: 1
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
JOIN TREE
TABLE_FUNCTION id: 8, table_function_name: numbers
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
@ -83,14 +83,14 @@ QUERY id: 0
LIST id: 3, nodes: 1
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
JOIN TREE
QUERY id: 5, is_subquery: 1
QUERY id: 5, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 6, nodes: 1
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
JOIN TREE
TABLE_FUNCTION id: 8, table_function_name: numbers
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
@ -119,14 +119,14 @@ QUERY id: 0
LIST id: 3, nodes: 1
COLUMN id: 4, column_name: x, result_type: UInt64, source_id: 5
JOIN TREE
QUERY id: 5, is_subquery: 1
QUERY id: 5, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 6, nodes: 1
COLUMN id: 7, column_name: number, result_type: UInt64, source_id: 8
JOIN TREE
TABLE_FUNCTION id: 8, table_function_name: numbers
TABLE_FUNCTION id: 8, alias: __table2, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
@ -171,7 +171,7 @@ QUERY id: 0
JOIN TREE
JOIN id: 8, strictness: ALL, kind: FULL
LEFT TABLE EXPRESSION
QUERY id: 3, alias: s, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
key UInt64
PROJECTION
@ -182,12 +182,12 @@ QUERY id: 0
COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13
CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 13, table_function_name: numbers
TABLE_FUNCTION id: 13, alias: __table2, table_function_name: numbers
ARGUMENTS
LIST id: 15, nodes: 1
CONSTANT id: 16, constant_value: UInt64_4, constant_value_type: UInt8
RIGHT TABLE EXPRESSION
TABLE id: 5, alias: t, table_name: default.test
TABLE id: 5, alias: __table3, table_name: default.test
JOIN EXPRESSION
LIST id: 17, nodes: 1
COLUMN id: 18, column_name: key, result_type: UInt64, source_id: 8
@ -220,7 +220,7 @@ QUERY id: 0
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.test
TABLE id: 3, alias: __table1, table_name: default.test
ORDER BY
LIST id: 5, nodes: 2
SORT id: 6, sort_direction: ASCENDING, with_fill: 0
@ -246,7 +246,7 @@ QUERY id: 0
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: a, result_type: UInt8, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.test
TABLE id: 3, alias: __table1, table_name: default.test
ORDER BY
LIST id: 5, nodes: 2
SORT id: 6, sort_direction: ASCENDING, with_fill: 0
@ -270,7 +270,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.test
TABLE id: 3, alias: __table1, table_name: default.test
GROUP BY
LIST id: 4, nodes: 1
COLUMN id: 2, column_name: key, result_type: UInt64, source_id: 3
@ -297,9 +297,9 @@ QUERY id: 0
JOIN TREE
JOIN id: 6, strictness: ALL, kind: INNER
LEFT TABLE EXPRESSION
TABLE id: 3, table_name: default.t1
TABLE id: 3, alias: __table1, table_name: default.t1
RIGHT TABLE EXPRESSION
TABLE id: 5, table_name: default.t2
TABLE id: 5, alias: __table2, table_name: default.t2
JOIN EXPRESSION
FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -1,9 +1,9 @@
===http===
{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"SELECT 1 AS `1` FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
{"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"SELECT 1 AS `1` FROM `system`.`one` AS `__table1`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"}
{"initial query spans with proper parent":"2"}

View File

@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1
"foo"
1
execute: --allow_experimental_analyzer=1 --stage fetch_columns
"dummy_0"
"__table1.dummy"
0
execute: --allow_experimental_analyzer=1 --stage with_mergeable_state
"1_UInt8"

View File

@ -917,9 +917,9 @@ from
;
Expression ((Project names + Projection))
Window (Window step for window \'\')
Window (Window step for window \'PARTITION BY p_0\')
Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
Window (Window step for window \'PARTITION BY __table1.p\')
Window (Window step for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\')
Sorting (Sorting for window \'PARTITION BY __table1.p ORDER BY __table1.o ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromSystemNumbers
explain select
@ -930,11 +930,11 @@ from
from numbers(16)) t
;
Expression ((Project names + Projection))
Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\')
Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\')
Window (Window step for window \'ORDER BY number_1 ASC\')
Window (Window step for window \'ORDER BY __table1.o ASC, __table1.number ASC\')
Sorting (Sorting for window \'ORDER BY __table1.o ASC, __table1.number ASC\')
Window (Window step for window \'ORDER BY __table1.number ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part])
Sorting (Sorting for window \'ORDER BY number_1 ASC\')
Sorting (Sorting for window \'ORDER BY __table1.number ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromSystemNumbers
-- A test case for the sort comparator found by fuzzer.

View File

@ -45,7 +45,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.constraint_test_constants
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
WHERE
FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -63,7 +63,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.constraint_test_constants
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
WHERE
FUNCTION id: 4, function_name: greater, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -80,5 +80,5 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.constraint_test_constants
TABLE id: 3, alias: __table1, table_name: default.constraint_test_constants
SETTINGS allow_experimental_analyzer=1

View File

@ -8,7 +8,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
WHERE
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
SETTINGS allow_experimental_analyzer=1
@ -22,7 +22,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
WHERE
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
SETTINGS allow_experimental_analyzer=1
@ -36,7 +36,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
WHERE
CONSTANT id: 4, constant_value: UInt64_0, constant_value_type: UInt8
SETTINGS allow_experimental_analyzer=1
@ -50,7 +50,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
WHERE
FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -68,7 +68,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
PREWHERE
FUNCTION id: 4, function_name: less, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -85,5 +85,5 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE id: 3, table_name: default.t_constraints_where
TABLE id: 3, alias: __table1, table_name: default.t_constraints_where
SETTINGS allow_experimental_analyzer=1

View File

@ -20,7 +20,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -50,7 +50,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
PREWHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -80,7 +80,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -110,7 +110,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -140,7 +140,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -162,7 +162,7 @@ QUERY id: 0
COLUMN id: 4, column_name: b, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_10, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.column_swap_test_test
TABLE id: 5, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -191,7 +191,7 @@ QUERY id: 0
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.column_swap_test_test
TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -223,7 +223,7 @@ QUERY id: 0
CONSTANT id: 8, constant_value: UInt64_10, constant_value_type: UInt8
COLUMN id: 9, column_name: a, result_type: String, source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.column_swap_test_test
TABLE id: 7, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -248,7 +248,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -270,7 +270,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -292,7 +292,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
COLUMN id: 4, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -310,7 +310,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.column_swap_test_test
TABLE id: 3, alias: __table1, table_name: default.column_swap_test_test
WHERE
FUNCTION id: 4, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -327,5 +327,5 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: UInt32, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_bad_constraint
TABLE id: 3, alias: __table1, table_name: default.t_bad_constraint
SETTINGS allow_experimental_analyzer=1

View File

@ -56,7 +56,7 @@ QUERY id: 0
CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 12, table_function_name: numbers
TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 15, nodes: 1
CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8
@ -82,7 +82,7 @@ QUERY id: 0
CONSTANT id: 13, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 14, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 12, table_function_name: numbers
TABLE_FUNCTION id: 12, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 15, nodes: 1
CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8
@ -111,7 +111,7 @@ QUERY id: 0
CONSTANT id: 15, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 16, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 14, table_function_name: numbers
TABLE_FUNCTION id: 14, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 17, nodes: 1
CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8

View File

@ -28,7 +28,7 @@ Aggregating
Filter
Filter
> (analyzer) filter should be pushed down after aggregating, column after aggregation is const
COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)
COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)
Aggregating
Filter
Filter
@ -49,9 +49,9 @@ Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is aliased
Filter column
ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))
ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(y_1, 0_UInt8)
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
1 2
2 3
@ -68,9 +68,9 @@ Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted
Filter column
FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2
FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2
Aggregating
Filter column: notEquals(y_1, 0_UInt8)
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
1 2
2 3
@ -87,9 +87,9 @@ Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
Filter column
FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))
FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(y_1, 0_UInt8)
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
1 2
2 3
@ -105,9 +105,9 @@ Aggregating
Filter column: and(notEquals(y, 0), minus(y, 4))
> (analyzer) two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased
Filter column
ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))
ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))
Aggregating
Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))
Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))
0 1
1 2
2 3
@ -121,9 +121,9 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0))
ARRAY JOIN x
Filter column: notEquals(y, 2)
> (analyzer) filter is split, one part is filtered before ARRAY JOIN
Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))
ARRAY JOIN x_0
Filter column: notEquals(y_1, 2_UInt8)
Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))
ARRAY JOIN __table1.x
Filter column: notEquals(__table2.y, 2_UInt8)
1 3
> filter is pushed down before Distinct
Distinct
@ -132,7 +132,7 @@ Filter column: notEquals(y, 2)
> (analyzer) filter is pushed down before Distinct
Distinct
Distinct
Filter column: notEquals(y_1, 2_UInt8)
Filter column: notEquals(__table1.y, 2_UInt8)
0 0
0 1
1 0
@ -144,7 +144,7 @@ Filter column: and(notEquals(x, 0), notEquals(y, 0))
> (analyzer) filter is pushed down before sorting steps
Sorting
Sorting
Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))
Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))
1 2
1 1
> filter is pushed down before TOTALS HAVING and aggregating
@ -154,7 +154,7 @@ Filter column: notEquals(y, 2)
> (analyzer) filter is pushed down before TOTALS HAVING and aggregating
TotalsHaving
Aggregating
Filter column: notEquals(y_0, 2_UInt8)
Filter column: notEquals(__table1.y, 2_UInt8)
0 12
1 15
3 10
@ -174,7 +174,7 @@ Join
> (analyzer) one condition of filter is pushed down before LEFT JOIN
Join
Join
Filter column: notEquals(number_0, 1_UInt8)
Filter column: notEquals(__table1.number, 1_UInt8)
0 0
3 3
> one condition of filter is pushed down before INNER JOIN
@ -185,7 +185,7 @@ Join
> (analyzer) one condition of filter is pushed down before INNER JOIN
Join
Join
Filter column: notEquals(number_0, 1_UInt8)
Filter column: notEquals(__table1.number, 1_UInt8)
3 3
> filter is pushed down before UNION
Union

View File

@ -36,7 +36,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from (
select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0
settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y_1, 0_UInt8)"
settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(__table1.y, 0_UInt8)"
$CLICKHOUSE_CLIENT -q "
select s, y, y != 0 from (select sum(x) as s, y from (
select number as x, number + 1 as y from numbers(10)) group by y
@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|ALIAS notEquals(s_0, 4_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|ALIAS notEquals(__table1.s, 4_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -76,7 +76,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 4_UInt8)) UInt8 : 2"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 4_UInt8) :: 0, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8)) UInt8 : 2"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -96,7 +96,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 8 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y_1, 0_UInt8)\|FUNCTION and(minus(s_0, 8_UInt8) :: 0, minus(s_0, 4_UInt8) :: 2) -> and(notEquals(y_1, 0_UInt8), minus(s_0, 8_UInt8), minus(s_0, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 0, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -116,7 +116,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s != 8 and y - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y_1, 0_UInt8), minus(y_1, 4_UInt8))\|ALIAS notEquals(s_0, 8_UInt8) :: 0 -> and(notEquals(y_1, 0_UInt8), notEquals(s_0, 8_UInt8), minus(y_1, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(__table1.y, 0_UInt8), minus(__table1.y, 4_UInt8))\|ALIAS notEquals(__table1.s, 8_UInt8) :: 0 -> and(notEquals(__table1.y, 0_UInt8), notEquals(__table1.s, 8_UInt8), minus(__table1.y, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
explain actions = 1 select x, y from (
select range(number) as x, number + 1 as y from numbers(3)
) array join x where y != 2 and x != 0" |
grep -o "Filter column: and(notEquals(y_1, 2_UInt8), notEquals(x_0, 0_UInt8))\|ARRAY JOIN x_0\|Filter column: notEquals(y_1, 2_UInt8)"
grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))\|ARRAY JOIN __table1.x\|Filter column: notEquals(__table2.y, 2_UInt8)"
$CLICKHOUSE_CLIENT -q "
select x, y from (
select range(number) as x, number + 1 as y from numbers(3)
@ -166,7 +166,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10))
) where y != 2
settings enable_optimize_predicate_expression=0" |
grep -o "Distinct\|Filter column: notEquals(y_1, 2_UInt8)"
grep -o "Distinct\|Filter column: notEquals(__table1.y, 2_UInt8)"
$CLICKHOUSE_CLIENT -q "
select x, y from (
select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10))
@ -186,7 +186,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --convert_query_to_cnf=0 -q "
select number % 2 as x, number % 3 as y from numbers(6) order by y desc
) where x != 0 and y != 0
settings enable_optimize_predicate_expression = 0" |
grep -o "Sorting\|Filter column: and(notEquals(x_0, 0_UInt8), notEquals(y_1, 0_UInt8))"
grep -o "Sorting\|Filter column: and(notEquals(__table1.x, 0_UInt8), notEquals(__table1.y, 0_UInt8))"
$CLICKHOUSE_CLIENT -q "
select x, y from (
select number % 2 as x, number % 3 as y from numbers(6) order by y desc
@ -206,7 +206,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals
) where y != 2
settings enable_optimize_predicate_expression=0" |
grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y_0, 2_UInt8)"
grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(__table1.y, 2_UInt8)"
$CLICKHOUSE_CLIENT -q "
select * from (
select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals
@ -236,7 +236,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select number as a, r.b from numbers(4) as l any left join (
select number + 2 as b from numbers(3)
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)"
grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)"
$CLICKHOUSE_CLIENT -q "
select number as a, r.b from numbers(4) as l any left join (
select number + 2 as b from numbers(3)
@ -255,7 +255,7 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
select number as a, r.b from numbers(4) as l any inner join (
select number + 2 as b from numbers(3)
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
grep -o "Join\|Filter column: notEquals(number_0, 1_UInt8)"
grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)"
$CLICKHOUSE_CLIENT -q "
select number as a, r.b from numbers(4) as l any inner join (
select number + 2 as b from numbers(3)

View File

@ -7,19 +7,19 @@ Partial sorting plan
Prefix sort description: n ASC
Result sort description: n ASC, x ASC
optimize_read_in_window_order=1, allow_experimental_analyzer=1
Prefix sort description: n_0 ASC
Result sort description: n_0 ASC, x_1 ASC
Prefix sort description: __table1.n ASC
Result sort description: __table1.n ASC, __table1.x ASC
No sorting plan
optimize_read_in_window_order=0
Sort description: n ASC, x ASC
optimize_read_in_window_order=0, allow_experimental_analyzer=1
Sort description: n_0 ASC, x_1 ASC
Sort description: __table1.n ASC, __table1.x ASC
optimize_read_in_window_order=1
Prefix sort description: n ASC, x ASC
Result sort description: n ASC, x ASC
optimize_read_in_window_order=1, allow_experimental_analyzer=1
Prefix sort description: n_0 ASC, x_1 ASC
Result sort description: n_0 ASC, x_1 ASC
Prefix sort description: __table1.n ASC, __table1.x ASC
Result sort description: __table1.n ASC, __table1.x ASC
Complex ORDER BY
optimize_read_in_window_order=0
3 3 1

View File

@ -37,59 +37,59 @@
"Node Type": "Aggregating",
"Header": [
{
"Name": "number_0",
"Name": "__table1.number",
"Type": "UInt64"
},
{
"Name": "quantile(0.2_Float64)(number_0)",
"Name": "quantile(0.2_Float64)(__table1.number)",
"Type": "Float64"
},
{
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
"Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))",
"Type": "UInt64"
}
],
"Keys": ["number_0"],
"Keys": ["__table1.number"],
"Aggregates": [
{
"Name": "quantile(0.2_Float64)(number_0)",
"Name": "quantile(0.2_Float64)(__table1.number)",
"Function": {
"Name": "quantile",
"Parameters": ["0.2"],
"Argument Types": ["UInt64"],
"Result Type": "Float64"
},
"Arguments": ["number_0"]
"Arguments": ["__table1.number"]
},
{
"Name": "sumIf(number_0, greater(number_0, 0_UInt8))",
"Name": "sumIf(__table1.number, greater(__table1.number, 0_UInt8))",
"Function": {
"Name": "sumIf",
"Argument Types": ["UInt64", "UInt8"],
"Result Type": "UInt64"
},
"Arguments": ["number_0", "greater(number_0, 0_UInt8)"]
"Arguments": ["__table1.number", "greater(__table1.number, 0_UInt8)"]
}
],
--------
"Node Type": "ArrayJoin",
"Left": false,
"Columns": ["x_0", "y_1"],
"Columns": ["__table1.x", "__table1.y"],
--------
"Node Type": "Distinct",
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
--
"Node Type": "Distinct",
"Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"],
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
--------
"Sort Description": [
{
"Column": "number_0",
"Column": "__table1.number",
"Ascending": false,
"With Fill": false
},
{
"Column": "plus(number_0, 1_UInt8)",
"Column": "plus(__table1.number, 1_UInt8)",
"Ascending": true,
"With Fill": false
}

View File

@ -30,7 +30,7 @@ SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key ==
SELECT '--';
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION }
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION,NOT_FOUND_COLUMN_IN_BLOCK }
SELECT '--';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2;

View File

@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1
"foo"
1
execute: --allow_experimental_analyzer=1 --stage fetch_columns
"dummy_0"
"__table1.dummy"
0
execute: --allow_experimental_analyzer=1 --stage with_mergeable_state
"1_UInt8"

View File

@ -11,7 +11,7 @@ QUERY id: 0
LIST id: 3, nodes: 1
CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String
JOIN TREE
TABLE id: 5, table_name: system.one
TABLE id: 5, alias: __table1, table_name: system.one
WHERE
FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -54,7 +54,7 @@ QUERY id: 0
LIST id: 3, nodes: 1
CONSTANT id: 4, constant_value: \'Привет, World\', constant_value_type: String
JOIN TREE
TABLE id: 5, table_name: system.one
TABLE id: 5, alias: __table1, table_name: system.one
WHERE
FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -4,15 +4,15 @@ EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255
Expression (Project names)
Header: avgWeighted(x, y) Nullable(Float64)
Expression (Projection)
Header: avgWeighted(x_0, y_1) Nullable(Float64)
Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64)
Aggregating
Header: avgWeighted(x_0, y_1) Nullable(Float64)
Header: avgWeighted(__table1.x, __table1.y) Nullable(Float64)
Expression (Before GROUP BY)
Header: x_0 Nullable(UInt8)
y_1 UInt8
Header: __table1.x Nullable(UInt8)
__table1.y UInt8
Expression (Change column names to column identifiers)
Header: x_0 Nullable(UInt8)
y_1 UInt8
Header: __table1.x Nullable(UInt8)
__table1.y UInt8
Union
Header: x Nullable(UInt8)
y UInt8
@ -26,7 +26,7 @@ Header: avgWeighted(x, y) Nullable(Float64)
Header: 255_UInt8 UInt8
1_UInt8 UInt8
Expression (Change column names to column identifiers)
Header: dummy_0 UInt8
Header: __table3.dummy UInt8
ReadFromStorage (SystemOne)
Header: dummy UInt8
Expression (Conversion before UNION)
@ -39,7 +39,7 @@ Header: avgWeighted(x, y) Nullable(Float64)
Header: NULL_Nullable(Nothing) Nullable(Nothing)
1_UInt8 UInt8
Expression (Change column names to column identifiers)
Header: dummy_0 UInt8
Header: __table5.dummy UInt8
ReadFromStorage (SystemOne)
Header: dummy UInt8
SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y);

View File

@ -2,35 +2,35 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query -
Expression ((Project names + Projection))
Header: dummy String
Aggregating
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
Expression ((Before GROUP BY + Change column names to column identifiers))
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
ReadFromStorage (SystemOne)
Header: dummy UInt8
clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy
Expression ((Project names + Projection))
Header: dummy String
Aggregating
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
Expression ((Before GROUP BY + Change column names to column identifiers))
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
ReadFromStorage (SystemOne)
Header: dummy UInt8
clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy
Expression ((Project names + Projection))
Header: dummy String
Aggregating
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
Expression ((Before GROUP BY + Change column names to column identifiers))
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
ReadFromStorage (SystemOne)
Header: dummy UInt8
clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy
Expression ((Project names + Projection))
Header: dummy String
Aggregating
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
Expression ((Before GROUP BY + Change column names to column identifiers))
Header: toString(dummy_0) String
Header: toString(__table1.dummy) String
ReadFromStorage (SystemOne)
Header: dummy UInt8

View File

@ -83,36 +83,36 @@ Sorting (Stream): a ASC, b ASC
Sorting (Stream): a ASC, b ASC
=== enable new analyzer ===
-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): a_1 ASC, b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, b ASC
-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns
Sorting (Stream): a_1 ASC
Sorting (Stream): a_1 ASC
Sorting (Stream): a_1 ASC
Sorting (Stream): __table1.a ASC
Sorting (Stream): __table1.a ASC
Sorting (Stream): __table1.a ASC
Sorting (Stream): a ASC
-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): a_1 ASC, b_0 ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): a ASC, b ASC
-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): a DESC, b DESC
-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause
Sorting (Stream): a_0 ASC, b_1 ASC
Sorting (Stream): a_0 ASC, b_1 ASC
Sorting (Stream): a_0 ASC, b_1 ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): a ASC, b ASC
-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): a_1 DESC, b_0 DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): __table1.a DESC, __table1.b DESC
Sorting (Stream): a DESC, b DESC
-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization
Sorting (Stream): a_0 ASC, b_1 ASC
Sorting (Stream): a_0 ASC, b_1 ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): __table1.a ASC, __table1.b ASC
Sorting (Stream): a ASC, b ASC

View File

@ -22,7 +22,7 @@ QUERY id: 0
COLUMN id: 2, column_name: id, result_type: UInt64, source_id: 3
COLUMN id: 4, column_name: value, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.test_table
TABLE id: 3, alias: __table1, table_name: default.test_table
--
QUERY id: 0
PROJECTION
@ -64,7 +64,7 @@ QUERY id: 0
CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8
CONSTANT id: 10, constant_value: Array_[UInt64_1, UInt64_2, UInt64_3], constant_value_type: Array(UInt8)
JOIN TREE
TABLE id: 11, table_name: default.test_table
TABLE id: 11, alias: __table1, table_name: default.test_table
--
QUERY id: 0
WITH
@ -99,4 +99,4 @@ QUERY id: 0
COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5
CONSTANT id: 6, constant_value: UInt64_1, constant_value_type: UInt8
JOIN TREE
TABLE id: 5, table_name: default.test_table
TABLE id: 5, alias: __table1, table_name: default.test_table

View File

@ -8,7 +8,7 @@ Sorting (None)
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a
Sorting (Global): a ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): a_0 ASC
Sorting (Global): __table1.a ASC
Sorting (None)
Sorting (None)
-- disable optimization -> sorting order is NOT propagated from subquery -> full sort
@ -36,8 +36,8 @@ Sorting (Stream): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a
Sorting (Global): a ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): a_0 ASC
Sorting (Stream): a_0 ASC
Sorting (Global): __table1.a ASC
Sorting (Stream): __table1.a ASC
Sorting (Stream): a ASC
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1
Sorting (None)
@ -48,8 +48,8 @@ Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1
Sorting (None)
Sorting (Sorting for ORDER BY)
Sorting (Global): plus(a_0, 1_UInt8) ASC
Sorting (Chunk): a_0 ASC
Sorting (Global): plus(__table1.a, 1_UInt8) ASC
Sorting (Chunk): __table1.a ASC
Sorting (Chunk): a ASC
-- ExpressionStep breaks sort mode
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1
@ -61,7 +61,7 @@ Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1
Sorting (Global): plus(a, 1) ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): plus(a_0, 1_UInt8) ASC
Sorting (Global): plus(__table1.a, 1_UInt8) ASC
Sorting (None)
Sorting (Chunk): a ASC
-- FilterStep preserves sort mode
@ -71,7 +71,7 @@ Sorting (Chunk): a ASC
Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0
Sorting (Chunk): a ASC
Sorting (Chunk): a_0 ASC
Sorting (Chunk): __table1.a ASC
Sorting (Chunk): a ASC
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0
Sorting (Chunk): a ASC
@ -79,7 +79,7 @@ Sorting (Chunk): a ASC
Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0
Sorting (Chunk): a ASC
Sorting (Chunk): a_0 ASC
Sorting (Chunk): __table1.a ASC
Sorting (Chunk): a ASC
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0
Sorting (Chunk): a ASC
@ -87,7 +87,7 @@ Sorting (Chunk): a ASC
Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0
Sorting (Chunk): a ASC
Sorting (Chunk): a_0 ASC
Sorting (Chunk): __table1.a ASC
Sorting (Chunk): a ASC
-- FilterStep breaks sort mode
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0
@ -119,11 +119,11 @@ Sorting (Stream): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a
Sorting (Global): a ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): a_0 ASC
Sorting (Global): __table1.a ASC
Sorting (None)
Sorting (Sorting for ORDER BY)
Sorting (Global): a_2 ASC
Sorting (Stream): a_2 ASC
Sorting (Global): __table3.a ASC
Sorting (Stream): __table3.a ASC
Sorting (Stream): a ASC
-- aliases DONT break sorting order
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)
@ -135,8 +135,8 @@ Sorting (Stream): a ASC, b ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)
Sorting (Global): a ASC, b ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): x_2 ASC, y_3 ASC
Sorting (Stream): x_2 ASC, y_3 ASC
Sorting (Global): __table2.x ASC, __table2.y ASC
Sorting (Stream): __table2.x ASC, __table2.y ASC
Sorting (Stream): a ASC, b ASC
-- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1)
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1
@ -151,11 +151,11 @@ Sorting (Chunk): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1
Sorting (None)
Sorting (Sorting for ORDER BY)
Sorting (Global): plus(a_0, 1_UInt8) ASC
Sorting (Global): plus(a_3, 1_UInt8) ASC
Sorting (Global): plus(__table1.a, 1_UInt8) ASC
Sorting (Global): plus(__table3.a, 1_UInt8) ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): plus(a_3, 1_UInt8) ASC
Sorting (Chunk): a_3 ASC
Sorting (Global): plus(__table3.a, 1_UInt8) ASC
Sorting (Chunk): __table3.a ASC
Sorting (Chunk): a ASC
-- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query
-- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a
@ -167,6 +167,6 @@ Sorting (Stream): a ASC
-- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a
Sorting (Global): a ASC
Sorting (Sorting for ORDER BY)
Sorting (Global): a_0 ASC
Sorting (Stream): a_0 ASC
Sorting (Global): __table1.a ASC
Sorting (Stream): __table1.a ASC
Sorting (Stream): a ASC

View File

@ -2,51 +2,51 @@ Expression
Header: key String
value String
Join
Header: key_0 String
value_1 String
Header: __table1.key String
__table3.value String
Expression
Header: key_0 String
Header: __table1.key String
ReadFromStorage
Header: dummy UInt8
Union
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
Expression
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
ReadFromStorage
Header: dummy UInt8
Expression
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
ReadFromStorage
Header: dummy UInt8
Expression
Header: key String
value String
Join
Header: key_0 String
key_2 String
value_1 String
Header: __table1.key String
__table3.key String
__table3.value String
Sorting
Header: key_0 String
Header: __table1.key String
Expression
Header: key_0 String
Header: __table1.key String
ReadFromStorage
Header: dummy UInt8
Sorting
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
Union
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
Expression
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
ReadFromStorage
Header: dummy UInt8
Expression
Header: key_2 String
value_1 String
Header: __table3.key String
__table3.value String
ReadFromStorage
Header: dummy UInt8

View File

@ -34,7 +34,7 @@ DROP TABLE t1;
SET allow_experimental_analyzer = 1;
SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number__ UInt64%';
SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: \_\_table1.number UInt64%';
SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%';
SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION);

View File

@ -4,19 +4,19 @@
2022-09-09 12:00:00 0x
2022-09-09 12:00:00 1
2022-09-09 12:00:00 1x
Prefix sort description: toStartOfMinute(t_0) ASC
Result sort description: toStartOfMinute(t_0) ASC, c1_1 ASC
Prefix sort description: toStartOfMinute(t_0) ASC
Result sort description: toStartOfMinute(t_0) ASC
Prefix sort description: negate(a_0) ASC
Result sort description: negate(a_0) ASC
Prefix sort description: negate(a_0) ASC, negate(b_1) ASC
Result sort description: negate(a_0) ASC, negate(b_1) ASC
Prefix sort description: a_0 DESC, negate(b_1) ASC
Result sort description: a_0 DESC, negate(b_1) ASC
Prefix sort description: negate(a_0) ASC, b_1 DESC
Result sort description: negate(a_0) ASC, b_1 DESC
Prefix sort description: negate(a_0) ASC
Result sort description: negate(a_0) ASC, b_1 ASC
Prefix sort description: a_0 ASC
Result sort description: a_0 ASC, negate(b_1) ASC
Prefix sort description: toStartOfMinute(__table1.t) ASC
Result sort description: toStartOfMinute(__table1.t) ASC, __table1.c1 ASC
Prefix sort description: toStartOfMinute(__table1.t) ASC
Result sort description: toStartOfMinute(__table1.t) ASC
Prefix sort description: negate(__table1.a) ASC
Result sort description: negate(__table1.a) ASC
Prefix sort description: negate(__table1.a) ASC, negate(__table1.b) ASC
Result sort description: negate(__table1.a) ASC, negate(__table1.b) ASC
Prefix sort description: __table1.a DESC, negate(__table1.b) ASC
Result sort description: __table1.a DESC, negate(__table1.b) ASC
Prefix sort description: negate(__table1.a) ASC, __table1.b DESC
Result sort description: negate(__table1.a) ASC, __table1.b DESC
Prefix sort description: negate(__table1.a) ASC
Result sort description: negate(__table1.a) ASC, __table1.b ASC
Prefix sort description: __table1.a ASC
Result sort description: __table1.a ASC, negate(__table1.b) ASC

View File

@ -21,7 +21,7 @@ QUERY id: 0
LIST id: 7, nodes: 1
COLUMN id: 4, column_name: a, result_type: Nullable(Int8), source_id: 5
JOIN TREE
TABLE id: 5, table_name: default.fuse_tbl
TABLE id: 5, alias: __table1, table_name: default.fuse_tbl
QUERY id: 0
PROJECTION COLUMNS
sum(b) Int64
@ -59,7 +59,7 @@ QUERY id: 0
COLUMN id: 6, column_name: b, result_type: Int8, source_id: 7
CONSTANT id: 18, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE id: 7, table_name: default.fuse_tbl
TABLE id: 7, alias: __table1, table_name: default.fuse_tbl
QUERY id: 0
PROJECTION COLUMNS
sum(plus(a, 1)) Nullable(Int64)
@ -138,7 +138,7 @@ QUERY id: 0
LIST id: 39, nodes: 1
COLUMN id: 6, column_name: a, result_type: Nullable(Int8), source_id: 7
JOIN TREE
TABLE id: 7, table_name: default.fuse_tbl
TABLE id: 7, alias: __table1, table_name: default.fuse_tbl
QUERY id: 0
PROJECTION COLUMNS
multiply(avg(b), 3) Float64
@ -215,14 +215,14 @@ QUERY id: 0
COLUMN id: 10, column_name: b, result_type: Int8, source_id: 11
CONSTANT id: 37, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
QUERY id: 11, is_subquery: 1
QUERY id: 11, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
b Int8
PROJECTION
LIST id: 38, nodes: 1
COLUMN id: 39, column_name: b, result_type: Int8, source_id: 40
JOIN TREE
TABLE id: 40, table_name: default.fuse_tbl
TABLE id: 40, alias: __table2, table_name: default.fuse_tbl
QUERY id: 0
PROJECTION COLUMNS
sum(b) Int64
@ -246,14 +246,14 @@ QUERY id: 0
COLUMN id: 6, column_name: b, result_type: Int64, source_id: 7
CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
QUERY id: 7, is_subquery: 1
QUERY id: 7, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
b Int64
PROJECTION
LIST id: 12, nodes: 1
COLUMN id: 13, column_name: x, result_type: Int64, source_id: 14
JOIN TREE
QUERY id: 14, is_subquery: 1
QUERY id: 14, alias: __table2, is_subquery: 1
PROJECTION COLUMNS
x Int64
count(b) UInt64
@ -276,7 +276,7 @@ QUERY id: 0
COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21
CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE id: 21, table_name: default.fuse_tbl
TABLE id: 21, alias: __table3, table_name: default.fuse_tbl
0 0 nan
0 0 nan
45 10 4.5 Decimal(38, 0) UInt64 Float64

View File

@ -34,7 +34,7 @@ QUERY id: 0
COLUMN id: 9, column_name: b, result_type: Float64, source_id: 10
CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
QUERY id: 10, is_subquery: 1
QUERY id: 10, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
b Float64
PROJECTION
@ -45,7 +45,7 @@ QUERY id: 0
COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19
CONSTANT id: 20, constant_value: UInt64_1, constant_value_type: UInt8
JOIN TREE
QUERY id: 19, is_subquery: 1
QUERY id: 19, alias: __table2, is_subquery: 1
PROJECTION COLUMNS
x Float64
quantile(0.9)(b) Float64
@ -76,7 +76,7 @@ QUERY id: 0
COLUMN id: 29, column_name: b, result_type: Int32, source_id: 30
CONSTANT id: 34, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE id: 30, table_name: default.fuse_tbl
TABLE id: 30, alias: __table3, table_name: default.fuse_tbl
GROUP BY
LIST id: 35, nodes: 1
COLUMN id: 18, column_name: x, result_type: Float64, source_id: 19

View File

@ -8,7 +8,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -26,7 +26,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -44,7 +44,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -62,7 +62,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -80,7 +80,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -106,7 +106,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality
TABLE id: 3, alias: __table1, table_name: default.t_logical_expressions_optimizer_low_cardinality
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS

Some files were not shown because too many files have changed in this diff Show More