Merge pull request #53705 from ClickHouse/analyzer-execution-names

Analyzer: always qualify execution names
This commit is contained in:
Nikolai Kochetov 2024-01-06 17:39:58 +01:00 committed by GitHub
commit 105789b936
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
67 changed files with 1026 additions and 695 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

@ -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");
@ -1058,6 +1058,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

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

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

View File

@ -24,4 +24,4 @@ SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id,
SELECT '--';
SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403 }
SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); -- { serverError 403, NOT_FOUND_COLUMN_IN_BLOCK }

View File

@ -50,7 +50,7 @@ QUERY id: 0
COLUMN id: 5, column_name: b, result_type: String, source_id: 3
COLUMN id: 6, column_name: c, result_type: String, source_id: 3
JOIN TREE
TABLE_FUNCTION id: 3, table_function_name: mysql
TABLE_FUNCTION id: 3, alias: __table1, table_function_name: mysql
ARGUMENTS
LIST id: 7, nodes: 5
CONSTANT id: 8, constant_value: \'127.0.0.1:9004\', constant_value_type: String
@ -63,10 +63,10 @@ QUERY id: 0
SETTINGS connection_wait_timeout=123 connect_timeout=40123002 read_write_timeout=40123001 connection_pool_size=3
SELECT
key AS key,
a AS a,
b AS b,
c AS c
FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3)
__table1.key AS key,
__table1.a AS a,
__table1.b AS b,
__table1.c AS c
FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) AS __table1
---
5

View File

@ -6,5 +6,5 @@
Order: a ASC, c ASC
ReadFromMergeTree (default.tab)
Aggregating
Order: a_0 ASC, c_2 ASC
Order: __table1.a ASC, __table1.c ASC
ReadFromMergeTree (default.tab)

View File

@ -20,7 +20,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_10, constant_value_type: UInt8
@ -44,7 +44,7 @@ QUERY id: 0
LIST id: 10, nodes: 1
CONSTANT id: 11, constant_value: UInt64_2, 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_10, constant_value_type: UInt8

View File

@ -17,7 +17,7 @@ QUERY id: 0, group_by_type: grouping_sets
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
@ -103,7 +103,7 @@ QUERY id: 0, group_by_type: grouping_sets
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
@ -180,7 +180,7 @@ QUERY id: 0, group_by_type: grouping_sets
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
@ -253,7 +253,7 @@ QUERY id: 0, group_by_type: grouping_sets
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
TABLE_FUNCTION id: 3, table_function_name: numbers
TABLE_FUNCTION id: 3, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 4, nodes: 1
CONSTANT id: 5, constant_value: UInt64_1000, constant_value_type: UInt16

View File

@ -16,7 +16,7 @@ QUERY id: 0
CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8
@ -41,7 +41,7 @@ QUERY id: 0
CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_10, constant_value_type: UInt8
@ -69,7 +69,7 @@ QUERY id: 0
CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8
CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 11, table_function_name: numbers
TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 14, nodes: 1
CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8

View File

@ -13,7 +13,7 @@ QUERY id: 0
LIST id: 6, nodes: 1
CONSTANT id: 7, constant_value: \'\', constant_value_type: String
JOIN TREE
TABLE_FUNCTION id: 8, table_function_name: numbers
TABLE_FUNCTION id: 8, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8

View File

@ -35,7 +35,7 @@ QUERY id: 0
CONSTANT id: 15, constant_value: \'other\', constant_value_type: String
CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String
JOIN TREE
TABLE id: 7, table_name: system.numbers
TABLE id: 7, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64
google
@ -78,7 +78,7 @@ QUERY id: 0
CONSTANT id: 17, constant_value: \'google\', constant_value_type: String
CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String
JOIN TREE
TABLE id: 9, table_name: system.numbers
TABLE id: 9, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64
other1
@ -122,7 +122,7 @@ QUERY id: 0
CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String
CONSTANT id: 19, constant_value: \'1\', constant_value_type: String
JOIN TREE
TABLE id: 9, table_name: system.numbers
TABLE id: 9, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64
google1
@ -169,7 +169,7 @@ QUERY id: 0
CONSTANT id: 20, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String
CONSTANT id: 21, constant_value: \'1\', constant_value_type: String
JOIN TREE
TABLE id: 11, table_name: system.numbers
TABLE id: 11, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64
google
@ -196,7 +196,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: String, source_id: 3
JOIN TREE
QUERY id: 3, alias: t1, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value String
PROJECTION
@ -223,7 +223,7 @@ QUERY id: 0
CONSTANT id: 20, constant_value: \'google\', constant_value_type: String
CONSTANT id: 21, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String
JOIN TREE
TABLE id: 12, table_name: system.numbers
TABLE id: 12, alias: __table2, table_name: system.numbers
LIMIT
CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64
other
@ -250,7 +250,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: String, source_id: 3
JOIN TREE
QUERY id: 3, alias: t1, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value String
PROJECTION
@ -274,7 +274,7 @@ QUERY id: 0
CONSTANT id: 18, constant_value: \'other\', constant_value_type: String
CONSTANT id: 19, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String
JOIN TREE
TABLE id: 10, table_name: system.numbers
TABLE id: 10, alias: __table2, table_name: system.numbers
LIMIT
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64
google google
@ -341,7 +341,7 @@ QUERY id: 0
CONSTANT id: 17, constant_value: \'google\', constant_value_type: String
CONSTANT id: 18, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2)\', constant_value_type: String
JOIN TREE
TABLE id: 9, table_name: system.numbers
TABLE id: 9, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64
other other
@ -402,7 +402,7 @@ QUERY id: 0
CONSTANT id: 15, constant_value: \'other\', constant_value_type: String
CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String
JOIN TREE
TABLE id: 7, table_name: system.numbers
TABLE id: 7, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64
other
@ -446,14 +446,14 @@ QUERY id: 0
CONSTANT id: 15, constant_value: \'other\', constant_value_type: String
CONSTANT id: 16, constant_value: \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\', constant_value_type: String
JOIN TREE
QUERY id: 7, is_subquery: 1
QUERY id: 7, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
number Nullable(Nothing)
PROJECTION
LIST id: 17, nodes: 1
CONSTANT id: 18, constant_value: NULL, constant_value_type: Nullable(Nothing)
JOIN TREE
TABLE id: 19, table_name: system.numbers
TABLE id: 19, alias: __table2, table_name: system.numbers
LIMIT
CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64
other
@ -482,7 +482,7 @@ QUERY id: 0
CONSTANT id: 7, constant_value: Array_[\'google\', \'censor.net\', \'yahoo\'], constant_value_type: Array(String)
CONSTANT id: 8, constant_value: \'other\', constant_value_type: String
JOIN TREE
TABLE id: 5, table_name: system.numbers
TABLE id: 5, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64
google
@ -514,6 +514,6 @@ QUERY id: 0
CONSTANT id: 9, constant_value: \'censor.net\', constant_value_type: String
CONSTANT id: 10, constant_value: \'google\', constant_value_type: String
JOIN TREE
TABLE id: 7, table_name: system.numbers
TABLE id: 7, alias: __table1, table_name: system.numbers
LIMIT
CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt64

View File

@ -12,7 +12,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value UInt64
PROJECTION
@ -23,7 +23,7 @@ QUERY id: 0
COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8
CONSTANT id: 9, constant_value: \'a\', constant_value_type: String
JOIN TREE
TABLE id: 8, table_name: default.test_table
TABLE id: 8, alias: __table2, table_name: default.test_table
SELECT '--';
--
EXPLAIN QUERY TREE SELECT value FROM (
@ -36,14 +36,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value UInt64
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table2, table_name: default.test_table
SETTINGS optimize_functions_to_subcolumns=1
SELECT '--';
--
@ -57,7 +57,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value UInt64
PROJECTION
@ -68,7 +68,7 @@ QUERY id: 0
COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8
CONSTANT id: 9, constant_value: \'a\', constant_value_type: String
JOIN TREE
TABLE id: 8, table_name: default.test_table
TABLE id: 8, alias: __table2, table_name: default.test_table
SETTINGS optimize_functions_to_subcolumns=0
SETTINGS optimize_functions_to_subcolumns=1
SELECT '--';
@ -83,7 +83,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value UInt64
PROJECTION
@ -94,7 +94,7 @@ QUERY id: 0
COLUMN id: 7, column_name: value, result_type: Tuple(a UInt64), source_id: 8
CONSTANT id: 9, constant_value: \'a\', constant_value_type: String
JOIN TREE
TABLE id: 8, table_name: default.test_table
TABLE id: 8, alias: __table2, table_name: default.test_table
SETTINGS optimize_functions_to_subcolumns=0
SELECT '--';
--
@ -108,13 +108,13 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value, result_type: UInt64, source_id: 3
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
value UInt64
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: value.a, result_type: UInt64, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table2, table_name: default.test_table
SETTINGS optimize_functions_to_subcolumns=1
SETTINGS optimize_functions_to_subcolumns=0

View File

@ -6,43 +6,43 @@ SELECT count() FROM a JOIN b ON b.b1 = a.a1 JOIN c ON c.c1 = b.b1 JOIN d ON d.d1
Expression ((Project names + Projection))
Header: count() UInt64
Aggregating
Header: a2_4 String
Header: __table1.a2 String
count() UInt64
Expression ((Before GROUP BY + DROP unused columns after JOIN))
Header: a2_4 String
Header: __table1.a2 String
Join (JOIN FillRightFirst)
Header: a2_4 String
c1_2 UInt64
Header: __table1.a2 String
__table3.c1 UInt64
Expression ((JOIN actions + DROP unused columns after JOIN))
Header: a2_4 String
c1_2 UInt64
Header: __table1.a2 String
__table3.c1 UInt64
Join (JOIN FillRightFirst)
Header: a2_4 String
b1_0 UInt64
c1_2 UInt64
Header: __table1.a2 String
__table2.b1 UInt64
__table3.c1 UInt64
Expression ((JOIN actions + DROP unused columns after JOIN))
Header: a2_4 String
b1_0 UInt64
Header: __table1.a2 String
__table2.b1 UInt64
Join (JOIN FillRightFirst)
Header: a1_1 UInt64
a2_4 String
b1_0 UInt64
Header: __table1.a1 UInt64
__table1.a2 String
__table2.b1 UInt64
Expression ((JOIN actions + Change column names to column identifiers))
Header: a1_1 UInt64
a2_4 String
Header: __table1.a1 UInt64
__table1.a2 String
ReadFromMemoryStorage
Header: a1 UInt64
a2 String
Expression ((JOIN actions + Change column names to column identifiers))
Header: b1_0 UInt64
Header: __table2.b1 UInt64
ReadFromMemoryStorage
Header: b1 UInt64
Expression ((JOIN actions + Change column names to column identifiers))
Header: c1_2 UInt64
Header: __table3.c1 UInt64
ReadFromMemoryStorage
Header: c1 UInt64
Expression ((JOIN actions + Change column names to column identifiers))
Header: d1_3 UInt64
Header: __table4.d1 UInt64
ReadFromMemoryStorage
Header: d1 UInt64
EXPLAIN PLAN header = 1
@ -52,38 +52,38 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN)))
Header: a2 String
d2 String
Join (JOIN FillRightFirst)
Header: a2_0 String
k_2 UInt64
d2_1 String
Header: __table1.a2 String
__table1.k UInt64
__table4.d2 String
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
Header: a2_0 String
k_2 UInt64
Header: __table1.a2 String
__table1.k UInt64
Join (JOIN FillRightFirst)
Header: a2_0 String
k_2 UInt64
Header: __table1.a2 String
__table1.k UInt64
Expression ((Actions for left table alias column keys + DROP unused columns after JOIN))
Header: a2_0 String
k_2 UInt64
Header: __table1.a2 String
__table1.k UInt64
Join (JOIN FillRightFirst)
Header: a2_0 String
k_2 UInt64
Header: __table1.a2 String
__table1.k UInt64
Expression ((Actions for left table alias column keys + Change column names to column identifiers))
Header: a2_0 String
k_2 UInt64
Header: __table1.a2 String
__table1.k UInt64
ReadFromMemoryStorage
Header: a2 String
k UInt64
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: k_5 UInt64
Header: __table2.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: k_4 UInt64
Header: __table3.k UInt64
ReadFromMemoryStorage
Header: k UInt64
Expression ((Actions for right table alias column keys + Change column names to column identifiers))
Header: d2_1 String
k_3 UInt64
Header: __table4.d2 String
__table4.k UInt64
ReadFromMemoryStorage
Header: d2 String
k UInt64
@ -97,55 +97,55 @@ WHERE c.c2 != '' ORDER BY a.a2
Expression (Project names)
Header: bx String
Sorting (Sorting for ORDER BY)
Header: a2_6 String
bx_0 String
Header: __table1.a2 String
__table2.bx String
Expression ((Before ORDER BY + (Projection + )))
Header: a2_6 String
bx_0 String
Header: __table1.a2 String
__table2.bx String
Join (JOIN FillRightFirst)
Header: a2_6 String
bx_0 String
c2_5 String
c1_3 UInt64
Header: __table1.a2 String
__table2.bx String
__table4.c2 String
__table4.c1 UInt64
Expression
Header: a2_6 String
bx_0 String
c2_5 String
c1_3 UInt64
Header: __table1.a2 String
__table2.bx String
__table4.c2 String
__table4.c1 UInt64
Join (JOIN FillRightFirst)
Header: a2_6 String
bx_0 String
b1_1 UInt64
c2_5 String
c1_3 UInt64
Header: __table1.a2 String
__table2.bx String
__table2.b1 UInt64
__table4.c2 String
__table4.c1 UInt64
Expression ((JOIN actions + DROP unused columns after JOIN))
Header: a2_6 String
bx_0 String
b1_1 UInt64
Header: __table1.a2 String
__table2.bx String
__table2.b1 UInt64
Join (JOIN FillRightFirst)
Header: a1_2 UInt64
a2_6 String
bx_0 String
b1_1 UInt64
Header: __table1.a1 UInt64
__table1.a2 String
__table2.bx String
__table2.b1 UInt64
Expression ((JOIN actions + Change column names to column identifiers))
Header: a1_2 UInt64
a2_6 String
Header: __table1.a1 UInt64
__table1.a2 String
ReadFromMemoryStorage
Header: a1 UInt64
a2 String
Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
Header: b1_1 UInt64
bx_0 String
Header: __table2.b1 UInt64
__table2.bx String
ReadFromMemoryStorage
Header: b1 UInt64
b2 String
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: c1_3 UInt64
c2_5 String
Header: __table4.c1 UInt64
__table4.c2 String
ReadFromMemoryStorage
Header: c1 UInt64
c2 String
Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
Header: d1_4 UInt64
Header: __table5.d1 UInt64
ReadFromSystemNumbers
Header: number UInt64

View File

@ -17,7 +17,7 @@ QUERY id: 0
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -40,7 +40,7 @@ QUERY id: 0
CONSTANT id: 11, constant_value: UInt64_0, constant_value_type: UInt8
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -63,7 +63,7 @@ QUERY id: 0
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing)
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -86,7 +86,7 @@ QUERY id: 0
CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing)
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -109,7 +109,7 @@ QUERY id: 0
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing)
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -132,7 +132,7 @@ QUERY id: 0
CONSTANT id: 11, constant_value: NULL, constant_value_type: Nullable(Nothing)
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 12, nodes: 1
CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8
@ -160,7 +160,7 @@ QUERY id: 0
COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13
CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing)
JOIN TREE
TABLE_FUNCTION id: 13, table_function_name: numbers
TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 16, nodes: 1
CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8
@ -188,7 +188,7 @@ QUERY id: 0
CONSTANT id: 15, constant_value: NULL, constant_value_type: Nullable(Nothing)
COLUMN id: 12, column_name: number, result_type: UInt64, source_id: 13
JOIN TREE
TABLE_FUNCTION id: 13, table_function_name: numbers
TABLE_FUNCTION id: 13, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 16, nodes: 1
CONSTANT id: 17, constant_value: UInt64_100, constant_value_type: UInt8
@ -207,7 +207,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8
@ -229,7 +229,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 11, nodes: 1
CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8
@ -248,7 +248,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8
@ -270,7 +270,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 11, nodes: 1
CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8
@ -289,7 +289,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 9, nodes: 1
CONSTANT id: 10, constant_value: UInt64_100, constant_value_type: UInt8
@ -311,7 +311,7 @@ QUERY id: 0
COLUMN id: 4, column_name: number, result_type: UInt64, source_id: 5
CONSTANT id: 10, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 5, table_function_name: numbers
TABLE_FUNCTION id: 5, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 11, nodes: 1
CONSTANT id: 12, constant_value: UInt64_100, constant_value_type: UInt8
@ -335,7 +335,7 @@ QUERY id: 0
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
CONSTANT id: 12, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 13, nodes: 1
CONSTANT id: 14, constant_value: UInt64_100, constant_value_type: UInt8
@ -362,7 +362,7 @@ QUERY id: 0
COLUMN id: 8, column_name: number, result_type: UInt64, source_id: 9
CONSTANT id: 14, constant_value: UInt64_2, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 9, table_function_name: numbers
TABLE_FUNCTION id: 9, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 15, nodes: 1
CONSTANT id: 16, constant_value: UInt64_100, constant_value_type: UInt8

View File

@ -16,7 +16,7 @@ QUERY id: 0
LIST id: 7, nodes: 1
COLUMN id: 8, column_name: value, result_type: String, source_id: 5
JOIN TREE
TABLE id: 5, table_name: default.test_table
TABLE id: 5, alias: __table1, table_name: default.test_table
GROUP BY
LIST id: 9, nodes: 2
COLUMN id: 4, column_name: id, result_type: UInt64, source_id: 5
@ -42,7 +42,7 @@ QUERY id: 0, group_by_type: rollup
COLUMN id: 9, column_name: __grouping_set, result_type: UInt64
COLUMN id: 10, column_name: value, result_type: String, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table1, table_name: default.test_table
GROUP BY
LIST id: 11, nodes: 2
COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6
@ -70,7 +70,7 @@ QUERY id: 0, group_by_type: cube
COLUMN id: 9, column_name: __grouping_set, result_type: UInt64
COLUMN id: 10, column_name: value, result_type: String, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table1, table_name: default.test_table
GROUP BY
LIST id: 11, nodes: 2
COLUMN id: 5, column_name: id, result_type: UInt64, source_id: 6
@ -99,7 +99,7 @@ QUERY id: 0, group_by_type: grouping_sets
COLUMN id: 9, column_name: __grouping_set, result_type: UInt64
COLUMN id: 10, column_name: value, result_type: String, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table1, table_name: default.test_table
GROUP BY
LIST id: 11, nodes: 2
LIST id: 12, nodes: 1
@ -128,7 +128,7 @@ QUERY id: 0, group_by_type: grouping_sets
COLUMN id: 9, column_name: __grouping_set, result_type: UInt64
COLUMN id: 10, column_name: value, result_type: String, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_table
TABLE id: 6, alias: __table1, table_name: default.test_table
GROUP BY
LIST id: 11, nodes: 2
LIST id: 12, nodes: 1

View File

@ -29,9 +29,9 @@ QUERY id: 0
LEFT TABLE EXPRESSION
JOIN id: 11, 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: 6, table_name: default.t2
TABLE id: 6, alias: __table2, table_name: default.t2
JOIN EXPRESSION
FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -48,14 +48,14 @@ QUERY id: 0
COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6
CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8
RIGHT TABLE EXPRESSION
QUERY id: 9, alias: t3, is_subquery: 1
QUERY id: 9, alias: __table3, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 23, nodes: 1
COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
JOIN TREE
TABLE id: 25, table_name: default.t3
TABLE id: 25, alias: __table4, table_name: default.t3
WHERE
FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -97,18 +97,18 @@ QUERY id: 0
LEFT TABLE EXPRESSION
JOIN id: 11, kind: COMMA
LEFT TABLE EXPRESSION
TABLE id: 3, table_name: default.t1
TABLE id: 3, alias: __table1, table_name: default.t1
RIGHT TABLE EXPRESSION
TABLE id: 6, table_name: default.t2
TABLE id: 6, alias: __table2, table_name: default.t2
RIGHT TABLE EXPRESSION
QUERY id: 9, alias: t3, is_subquery: 1
QUERY id: 9, alias: __table3, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 12, nodes: 1
COLUMN id: 13, column_name: a, result_type: UInt64, source_id: 14
JOIN TREE
TABLE id: 14, table_name: default.t3
TABLE id: 14, alias: __table4, table_name: default.t3
WHERE
FUNCTION id: 15, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -166,9 +166,9 @@ QUERY id: 0
LEFT TABLE EXPRESSION
JOIN id: 11, 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: 6, table_name: default.t2
TABLE id: 6, alias: __table2, table_name: default.t2
JOIN EXPRESSION
FUNCTION id: 12, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -185,14 +185,14 @@ QUERY id: 0
COLUMN id: 21, column_name: a, result_type: UInt64, source_id: 6
CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8
RIGHT TABLE EXPRESSION
QUERY id: 9, alias: t3, is_subquery: 1
QUERY id: 9, alias: __table3, is_subquery: 1
PROJECTION COLUMNS
x UInt64
PROJECTION
LIST id: 23, nodes: 1
COLUMN id: 24, column_name: a, result_type: UInt64, source_id: 25
JOIN TREE
TABLE id: 25, table_name: default.t3
TABLE id: 25, alias: __table4, table_name: default.t3
WHERE
FUNCTION id: 26, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -1,21 +1,21 @@
Expression ((Project names + (Projection + )))
Header: number UInt64
Actions: INPUT : 0 -> number_1 UInt64 : 0
ALIAS number_1 :: 0 -> number UInt64 : 1
ALIAS number :: 1 -> number_0 UInt64 : 0
ALIAS number_0 :: 0 -> number UInt64 : 1
Actions: INPUT : 0 -> __table2.number UInt64 : 0
ALIAS __table2.number :: 0 -> number UInt64 : 1
ALIAS number :: 1 -> __table1.number UInt64 : 0
ALIAS __table1.number :: 0 -> number UInt64 : 1
Positions: 1
Sorting (Sorting for ORDER BY)
Header: ignore(2_UInt8) UInt8
number_1 UInt64
__table2.number UInt64
Sort description: ignore(2_UInt8) ASC
Filter (( + (Before ORDER BY + (Projection + Change column names to column identifiers))))
Header: ignore(2_UInt8) UInt8
number_1 UInt64
__table2.number UInt64
Filter column: ignore(2_UInt8)
Actions: INPUT : 0 -> number UInt64 : 0
COLUMN Const(UInt8) -> 2_UInt8 UInt8 : 1
ALIAS number :: 0 -> number_1 UInt64 : 2
ALIAS number :: 0 -> __table2.number UInt64 : 2
FUNCTION ignore(2_UInt8 :: 1) -> ignore(2_UInt8) UInt8 : 0
Positions: 0 2
ReadFromSystemNumbers

View File

@ -26,7 +26,7 @@ QUERY id: 0
LIST id: 14, nodes: 1
CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 16, table_function_name: numbers
TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 17, nodes: 1
CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8
@ -58,7 +58,7 @@ QUERY id: 0
LIST id: 14, nodes: 1
CONSTANT id: 15, constant_value: UInt64_10, constant_value_type: UInt8
JOIN TREE
TABLE_FUNCTION id: 16, table_function_name: numbers
TABLE_FUNCTION id: 16, alias: __table1, table_function_name: numbers
ARGUMENTS
LIST id: 17, nodes: 1
CONSTANT id: 18, constant_value: UInt64_10, constant_value_type: UInt8
@ -81,7 +81,7 @@ QUERY id: 0
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8
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_10, constant_value_type: UInt8
@ -104,7 +104,7 @@ QUERY id: 0
CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8
CONSTANT id: 10, constant_value: UInt64_5, constant_value_type: UInt8
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_10, constant_value_type: UInt8

View File

@ -9,7 +9,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -26,7 +26,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -42,7 +42,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
CONSTANT id: 5, constant_value: UInt64_0, constant_value_type: UInt8
3 another
@ -55,7 +55,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -80,7 +80,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -97,7 +97,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -115,7 +115,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02668_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02668_logical_optimizer
WHERE
FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -2,31 +2,31 @@ Expression ((Project names + (Projection + )))
Header: t1.id UInt64
t1.value String
t2.value String
Actions: INPUT : 0 -> id_0 UInt64 : 0
INPUT : 1 -> value_1 String : 1
INPUT : 2 -> value_2 String : 2
ALIAS id_0 :: 0 -> t1.id UInt64 : 3
ALIAS value_1 :: 1 -> t1.value String : 0
ALIAS value_2 :: 2 -> t2.value String : 1
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
ALIAS __table1.id :: 0 -> t1.id UInt64 : 3
ALIAS __table1.value :: 1 -> t1.value String : 0
ALIAS __table2.value :: 2 -> t2.value String : 1
Positions: 3 0 1
FilledJoin (Filled JOIN)
Header: id_0 UInt64
value_1 String
value_2 String
Header: __table1.id UInt64
__table1.value String
__table2.value String
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(id_0) = (id)]
Clauses: [(__table1.id) = (id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: id_0 UInt64
value_1 String
Filter column: equals(id_0, 0_UInt8) (removed)
Header: __table1.id UInt64
__table1.value String
Filter column: equals(__table1.id, 0_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2
ALIAS id :: 0 -> id_0 UInt64 : 3
ALIAS value :: 1 -> value_1 String : 0
FUNCTION equals(id_0 : 3, 0_UInt8 :: 2) -> equals(id_0, 0_UInt8) UInt8 : 1
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 0_UInt8 :: 2) -> equals(__table1.id, 0_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table)
Header: id UInt64

View File

@ -29,10 +29,10 @@ Header: id UInt64
value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> id_0 UInt64 : 2
ALIAS value :: 1 -> value_1 String : 0
ALIAS id_0 :: 2 -> id UInt64 : 1
ALIAS value_1 :: 0 -> value String : 2
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
ALIAS __table1.id :: 2 -> id UInt64 : 1
ALIAS __table1.value :: 0 -> value String : 2
Positions: 1 2
ReadFromMergeTree (default.test_table)
Header: id UInt64

View File

@ -9,7 +9,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02702_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer
WHERE
FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8)
ARGUMENTS
@ -41,7 +41,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02702_logical_optimizer
TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer
WHERE
FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: Nullable(UInt8)
ARGUMENTS
@ -68,7 +68,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Nullable(Int32), source_id: 3
COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02702_logical_optimizer_with_null_column
TABLE id: 3, alias: __table1, table_name: default.02702_logical_optimizer_with_null_column
WHERE
FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: Nullable(UInt8)
ARGUMENTS

View File

@ -9,4 +9,4 @@
7885388429666205427
8124171311239967992
1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1;
0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1
0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1

View File

@ -8,7 +8,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -50,7 +50,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -92,7 +92,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -126,7 +126,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -160,7 +160,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -194,7 +194,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -228,7 +228,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -270,7 +270,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -335,7 +335,7 @@ QUERY id: 0
LIST id: 5, nodes: 1
COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 7, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -377,7 +377,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -412,7 +412,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
PREWHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -452,7 +452,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -492,7 +492,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -529,7 +529,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -566,7 +566,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -608,7 +608,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -650,7 +650,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -692,7 +692,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -726,7 +726,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -760,7 +760,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -794,7 +794,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -828,7 +828,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date_t
TABLE id: 3, alias: __table1, table_name: default.date_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -878,7 +878,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.datetime_t
TABLE id: 3, alias: __table1, table_name: default.datetime_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -920,7 +920,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.datetime_t
TABLE id: 3, alias: __table1, table_name: default.datetime_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -962,7 +962,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date32_t
TABLE id: 3, alias: __table1, table_name: default.date32_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -1004,7 +1004,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.date32_t
TABLE id: 3, alias: __table1, table_name: default.date32_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -1046,7 +1046,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.datetime64_t
TABLE id: 3, alias: __table1, table_name: default.datetime64_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -1088,7 +1088,7 @@ QUERY id: 0
LIST id: 1, nodes: 1
COLUMN id: 2, column_name: value1, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.datetime64_t
TABLE id: 3, alias: __table1, table_name: default.datetime64_t
WHERE
FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS

View File

@ -3,31 +3,31 @@ Header: id UInt64
value_1 String
rhs.id UInt64
rhs.value_1 String
Actions: INPUT : 0 -> id_0 UInt64 : 0
INPUT : 1 -> value_1_1 String : 1
INPUT : 2 -> value_1_3 String : 2
INPUT : 3 -> id_2 UInt64 : 3
ALIAS id_0 :: 0 -> id UInt64 : 4
ALIAS value_1_1 :: 1 -> value_1 String : 0
ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1
ALIAS id_2 :: 3 -> rhs.id UInt64 : 2
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value_1 String : 1
INPUT : 2 -> __table2.value_1 String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value_1 :: 1 -> value_1 String : 0
ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 0 2 1
Join (JOIN FillRightFirst)
Header: id_0 UInt64
value_1_1 String
value_1_3 String
id_2 UInt64
Header: __table1.id UInt64
__table1.value_1 String
__table2.value_1 String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(id_0) = (id_2)]
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: id_0 UInt64
value_1_1 String
Header: __table1.id UInt64
__table1.value_1 String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value_1 String : 1
ALIAS id :: 0 -> id_0 UInt64 : 2
ALIAS value_1 :: 1 -> value_1_1 String : 0
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value_1 :: 1 -> __table1.value_1 String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
@ -36,12 +36,12 @@ Positions: 4 0 2 1
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: id_2 UInt64
value_1_3 String
Header: __table2.id UInt64
__table2.value_1 String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value_1 String : 1
ALIAS id :: 0 -> id_2 UInt64 : 2
ALIAS value_1 :: 1 -> value_1_3 String : 0
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value_1 :: 1 -> __table2.value_1 String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
@ -55,39 +55,39 @@ Header: id UInt64
value_1 String
rhs.id UInt64
rhs.value_1 String
Actions: INPUT : 0 -> id_0 UInt64 : 0
INPUT : 1 -> value_1_1 String : 1
INPUT :: 2 -> value_2_4 UInt64 : 2
INPUT : 3 -> value_1_3 String : 3
INPUT :: 4 -> value_2_5 UInt64 : 4
INPUT : 5 -> id_2 UInt64 : 5
ALIAS id_0 :: 0 -> id UInt64 : 6
ALIAS value_1_1 :: 1 -> value_1 String : 0
ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1
ALIAS id_2 :: 5 -> rhs.id UInt64 : 3
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value_1 String : 1
INPUT :: 2 -> __table1.value_2 UInt64 : 2
INPUT : 3 -> __table2.value_1 String : 3
INPUT :: 4 -> __table2.value_2 UInt64 : 4
INPUT : 5 -> __table2.id UInt64 : 5
ALIAS __table1.id :: 0 -> id UInt64 : 6
ALIAS __table1.value_1 :: 1 -> value_1 String : 0
ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1
ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3
Positions: 6 0 3 1
Join (JOIN FillRightFirst)
Header: id_0 UInt64
value_1_1 String
value_2_4 UInt64
value_1_3 String
value_2_5 UInt64
id_2 UInt64
Header: __table1.id UInt64
__table1.value_1 String
__table1.value_2 UInt64
__table2.value_1 String
__table2.value_2 UInt64
__table2.id UInt64
Type: INNER
Strictness: ASOF
Algorithm: HashJoin
ASOF inequality: LESS
Clauses: [(id_0, value_2_4) = (id_2, value_2_5)]
Clauses: [(__table1.id, __table1.value_2) = (__table2.id, __table2.value_2)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: id_0 UInt64
value_1_1 String
value_2_4 UInt64
Header: __table1.id UInt64
__table1.value_1 String
__table1.value_2 UInt64
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value_1 String : 1
INPUT : 2 -> value_2 UInt64 : 2
ALIAS id :: 0 -> id_0 UInt64 : 3
ALIAS value_1 :: 1 -> value_1_1 String : 0
ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value_1 :: 1 -> __table1.value_1 String : 0
ALIAS value_2 :: 2 -> __table1.value_2 UInt64 : 1
Positions: 3 0 1
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
@ -97,15 +97,15 @@ Positions: 6 0 3 1
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: id_2 UInt64
value_1_3 String
value_2_5 UInt64
Header: __table2.id UInt64
__table2.value_1 String
__table2.value_2 UInt64
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value_1 String : 1
INPUT : 2 -> value_2 UInt64 : 2
ALIAS id :: 0 -> id_2 UInt64 : 3
ALIAS value_1 :: 1 -> value_1_3 String : 0
ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1
ALIAS id :: 0 -> __table2.id UInt64 : 3
ALIAS value_1 :: 1 -> __table2.value_1 String : 0
ALIAS value_2 :: 2 -> __table2.value_2 UInt64 : 1
Positions: 3 0 1
ReadFromMergeTree (default.test_table_2)
Header: id UInt64

View File

@ -15,14 +15,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, is_subquery: 1, is_distinct: 1
QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1
PROJECTION COLUMNS
a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
SETTINGS allow_experimental_analyzer=1
2. test distinct with subquery alias
3
@ -41,14 +41,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1
QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1
PROJECTION COLUMNS
a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
SETTINGS allow_experimental_analyzer=1
3. test distinct with compound column name
3
@ -67,14 +67,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1
QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1
PROJECTION COLUMNS
a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
SETTINGS allow_experimental_analyzer=1
4. test distinct with select expression alias
3
@ -93,14 +93,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1, is_distinct: 1
QUERY id: 3, alias: __table1, is_subquery: 1, is_distinct: 1
PROJECTION COLUMNS
alias_of_a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
SETTINGS allow_experimental_analyzer=1
5. test simple group by
3
@ -122,14 +122,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
GROUP BY
LIST id: 7, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
@ -154,14 +154,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
GROUP BY
LIST id: 7, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
@ -186,14 +186,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
alias_of_a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
GROUP BY
LIST id: 7, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
@ -218,14 +218,14 @@ QUERY id: 0
LIST id: 1, nodes: 1
FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64
JOIN TREE
QUERY id: 3, alias: t, is_subquery: 1
QUERY id: 3, alias: __table1, is_subquery: 1
PROJECTION COLUMNS
alias_of_a UInt8
PROJECTION
LIST id: 4, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6
JOIN TREE
TABLE id: 6, table_name: default.test_rewrite_uniq_to_count
TABLE id: 6, alias: __table2, table_name: default.test_rewrite_uniq_to_count
GROUP BY
LIST id: 7, nodes: 1
COLUMN id: 5, column_name: a, result_type: UInt8, source_id: 6

View File

@ -13,8 +13,8 @@ select * from tab order by (a + b) * c;
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from tab order by (a + b) * c) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
select * from tab order by (a + b) * c desc;
4 4 4 4
4 4 4 4
@ -27,8 +27,8 @@ select * from tab order by (a + b) * c desc;
0 0 0 0
0 0 0 0
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC
-- Exact match, full key
select * from tab order by (a + b) * c, sin(a / b);
0 0 0 0
@ -42,8 +42,8 @@ select * from tab order by (a + b) * c, sin(a / b);
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
select * from tab order by (a + b) * c desc, sin(a / b) desc;
4 4 4 4
4 4 4 4
@ -56,8 +56,8 @@ select * from tab order by (a + b) * c desc, sin(a / b) desc;
0 0 0 0
0 0 0 0
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b) desc) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) DESC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) DESC
-- Exact match, mixed direction
select * from tab order by (a + b) * c desc, sin(a / b);
4 4 4 4
@ -71,8 +71,8 @@ select * from tab order by (a + b) * c desc, sin(a / b);
0 0 0 0
0 0 0 0
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, sin(divide(__table1.a, __table1.b)) ASC
select * from tab order by (a + b) * c, sin(a / b) desc;
0 0 0 0
0 0 0 0
@ -85,8 +85,8 @@ select * from tab order by (a + b) * c, sin(a / b) desc;
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b) desc) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) DESC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) DESC
-- Wrong order, full sort
select * from tab order by sin(a / b), (a + b) * c;
1 1 1 1
@ -100,32 +100,32 @@ select * from tab order by sin(a / b), (a + b) * c;
0 0 0 0
0 0 0 0
select * from (explain plan actions = 1 select * from tab order by sin(a / b), (a + b) * c) where explain ilike '%sort description%';
Sort description: sin(divide(a_0, b_1)) ASC, multiply(plus(a_0, b_1), c_2) ASC
Sort description: sin(divide(__table1.a, __table1.b)) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC
-- Fixed point
select * from tab where (a + b) * c = 8 order by sin(a / b);
2 2 2 2
2 2 2 2
select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%';
Prefix sort description: sin(divide(a_0, b_1)) ASC
Result sort description: sin(divide(a_0, b_1)) ASC
Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC
Result sort description: sin(divide(__table1.a, __table1.b)) ASC
select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c;
1 1 1 1
1 1 1 1
select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c) where explain ilike '%sort description%';
Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, multiply(plus(a_0, b_1), c_2) ASC
Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, multiply(plus(__table1.a, __table1.b), __table1.c) ASC
select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b);
2 2 2 2
2 2 2 2
select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b)) where explain ilike '%sort description%';
Prefix sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(d_3, 1_UInt8), 4_UInt8) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.d, 1_UInt8), 4_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC
-- Wrong order with fixed point
select * from tab where (a + b) * c = 8 order by sin(b / a);
2 2 2 2
2 2 2 2
select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(b / a)) where explain ilike '%sort description%';
Sort description: sin(divide(b_1, a_0)) ASC
Sort description: sin(divide(__table1.b, __table1.a)) ASC
-- Monotonicity
select * from tab order by intDiv((a + b) * c, 2);
0 0 0 0
@ -139,8 +139,8 @@ select * from tab order by intDiv((a + b) * c, 2);
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2)) where explain like '%sort description%';
Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC
Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC
Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC
Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC
select * from tab order by intDiv((a + b) * c, 2), sin(a / b);
0 0 0 0
0 0 0 0
@ -153,36 +153,36 @@ select * from tab order by intDiv((a + b) * c, 2), sin(a / b);
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2), sin(a / b)) where explain like '%sort description%';
Prefix sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC
Result sort description: intDiv(multiply(plus(a_0, b_1), c_2), 2_UInt8) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC
Result sort description: intDiv(multiply(plus(__table1.a, __table1.b), __table1.c), 2_UInt8) ASC, sin(divide(__table1.a, __table1.b)) ASC
-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2);
select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC
-- select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2);
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) ASC
-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc;
select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC
-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc;
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), 2_UInt8) DESC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), 2_UInt8) DESC
-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2);
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(sin(divide(a_0, b_1)), -2_Int8) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8) ASC
-- select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3);
select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) DESC
Result sort description: multiply(plus(a_0, b_1), c_2) DESC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) DESC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC
-- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3);
select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, intDiv(intDiv(sin(divide(a_0, b_1)), -2_Int8), -3_Int8) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, intDiv(intDiv(sin(divide(__table1.a, __table1.b)), -2_Int8), -3_Int8) ASC
-- Aliases
select * from (select *, a + b as x from tab) order by x * c;
0 0 0 0 0
@ -196,8 +196,8 @@ select * from (select *, a + b as x from tab) order by x * c;
4 4 4 4 8
4 4 4 4 8
select * from (explain plan actions = 1 select * from (select *, a + b as x from tab) order by x * c) where explain like '%sort description%';
Prefix sort description: multiply(x_4, c_2) ASC
Result sort description: multiply(x_4, c_2) ASC
Prefix sort description: multiply(__table1.x, __table1.c) ASC
Result sort description: multiply(__table1.x, __table1.c) ASC
select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y);
0 0 0 0 0 nan
0 0 0 0 0 nan
@ -210,8 +210,8 @@ select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)
4 4 4 4 8 1
4 4 4 4 8 1
select * from (explain plan actions = 1 select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)) where explain like '%sort description%';
Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC
Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC
Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC
Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC
select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y);
0 0 0 0 0 nan
0 0 0 0 0 nan
@ -224,8 +224,8 @@ select * from (select *, a / b as y from (select *, a + b as x from tab)) order
4 4 4 4 8 1
4 4 4 4 8 1
select * from (explain plan actions = 1 select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y)) where explain like '%sort description%';
Prefix sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC
Result sort description: multiply(x_4, c_2) ASC, sin(y_5) ASC
Prefix sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC
Result sort description: multiply(__table1.x, __table1.c) ASC, sin(__table1.y) ASC
-- { echoOn }
select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3);
@ -238,8 +238,8 @@ select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intD
2020-02-05 00:00:00 3 3
2020-02-05 00:00:00 3 3
select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%';
Prefix sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC
Result sort description: toTimezone(toTimezone(x_0, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC
Prefix sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC
Result sort description: toTimezone(toTimezone(__table1.x, \'UTC\'_String), \'CET\'_String) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC
select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3);
2020-02-02 00:00:00 0 0
2020-02-02 00:00:00 0 0
@ -250,12 +250,12 @@ select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3);
2020-02-05 00:00:00 3 3
2020-02-05 00:00:00 3 3
select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%';
Prefix sort description: toStartOfDay(x_0) ASC
Result sort description: toStartOfDay(x_0) ASC, intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC
Prefix sort description: toStartOfDay(__table1.x) ASC
Result sort description: toStartOfDay(__table1.x) ASC, intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC
-- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3);
select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%';
Prefix sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC
Result sort description: intDiv(intDiv(y_1, -2_Int8), -3_Int8) ASC
Prefix sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC
Result sort description: intDiv(intDiv(__table1.y, -2_Int8), -3_Int8) ASC
-- { echoOn }
-- Union (not fully supported)
@ -281,8 +281,8 @@ select * from (select * from tab union all select * from tab3) order by (a + b)
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b);
@ -291,8 +291,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
3 3 3 3
3 3 3 3
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: sin(divide(a_0, b_1)) ASC
Result sort description: sin(divide(a_0, b_1)) ASC
Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC
Result sort description: sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b);
@ -309,8 +309,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
0 0 0 0
0 0 0 0
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: sin(divide(a_0, b_1)) ASC
Result sort description: sin(divide(a_0, b_1)) ASC
Prefix sort description: sin(divide(__table1.a, __table1.b)) ASC
Result sort description: sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab union all select * from tab5) order by (a + b) * c;
@ -335,8 +335,8 @@ select * from (select * from tab union all select * from tab5) order by (a + b)
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b);
@ -361,11 +361,11 @@ select * from (select * from tab union all select * from tab5) order by (a + b)
4 4 4 4
4 4 4 4
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
-- Union with limit
select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3;
@ -375,12 +375,12 @@ select * from (select * from tab union all select * from tab5) order by (a + b)
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%';
Limit (preliminary LIMIT (without OFFSET))
Limit 3
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Limit 3
ReadType: InOrder
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
-- In this example, we read-in-order from tab up to ((a + b) * c, sin(a / b)) and from tab5 up to ((a + b) * c).
-- In case of tab5, there would be two finish sorting transforms: ((a + b) * c) -> ((a + b) * c, sin(a / b)) -> ((a + b) * c, sin(a / b), d).
@ -393,14 +393,14 @@ select * from (select * from tab union all select * from tab5 union all select *
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%';
Limit (preliminary LIMIT (without OFFSET))
Limit 3
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC
Limit 3
ReadType: InOrder
Prefix sort description: multiply(plus(a_0, b_1), c_2) ASC
Result sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC
Prefix sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC
Result sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC
ReadType: InOrder
Sort description: multiply(plus(a_0, b_1), c_2) ASC, sin(divide(a_0, b_1)) ASC, d_3 ASC
Sort description: multiply(plus(__table1.a, __table1.b), __table1.c) ASC, sin(divide(__table1.a, __table1.b)) ASC, __table1.d ASC
Limit 3
ReadType: Default
drop table if exists tab;

View File

@ -9,7 +9,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02952_disjunction_optimization
TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization
WHERE
FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -27,7 +27,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02952_disjunction_optimization
TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: Bool
ARGUMENTS
@ -48,7 +48,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02952_disjunction_optimization
TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -73,7 +73,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02952_disjunction_optimization
TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization
WHERE
FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8
ARGUMENTS
@ -100,7 +100,7 @@ QUERY id: 0
COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3
COLUMN id: 4, column_name: b, result_type: String, source_id: 3
JOIN TREE
TABLE id: 3, table_name: default.02952_disjunction_optimization
TABLE id: 3, alias: __table1, table_name: default.02952_disjunction_optimization
WHERE
FUNCTION id: 5, function_name: or, function_type: ordinary, result_type: UInt8
ARGUMENTS