From 40cb904a3d5d85e35d718f433d521a924fd4ad95 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Aug 2022 18:34:10 +0200 Subject: [PATCH] Added support for JOIN, ARRAY JOIN --- src/Analyzer/ArrayJoinNode.cpp | 76 + src/Analyzer/ArrayJoinNode.h | 105 ++ src/Analyzer/ColumnNode.cpp | 48 +- src/Analyzer/ColumnNode.h | 45 +- src/Analyzer/IQueryTreeNode.cpp | 2 + src/Analyzer/IQueryTreeNode.h | 2 + src/Analyzer/JoinNode.cpp | 159 ++ src/Analyzer/JoinNode.h | 148 ++ src/Analyzer/ListNode.cpp | 5 + src/Analyzer/ListNode.h | 6 + src/Analyzer/MatcherNode.cpp | 2 + src/Analyzer/QueryAnalysisPass.cpp | 1225 +++++++----- src/Analyzer/QueryNode.cpp | 56 +- src/Analyzer/QueryNode.h | 8 + src/Analyzer/QueryTreeBuilder.cpp | 93 +- src/Analyzer/TableNode.cpp | 8 +- src/Analyzer/TableNode.h | 16 +- src/Analyzer/Utils.cpp | 94 + src/Analyzer/Utils.h | 8 + src/Analyzer/examples/query_analyzer.cpp | 2 - src/Interpreters/ExpressionActions.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 28 +- .../InterpreterSelectQueryAnalyzer.cpp | 1636 +++++++++++++++-- .../InterpreterSelectQueryAnalyzer.h | 7 +- src/Interpreters/TableJoin.cpp | 153 +- src/Interpreters/TableJoin.h | 44 +- src/Interpreters/TreeRewriter.cpp | 17 +- .../02337_analyzer_columns_basic.sql | 1 + ...369_analyzer_array_join_function.reference | 38 - .../02369_analyzer_array_join_function.sql | 9 +- .../02371_analyzer_join_cross.reference | 133 ++ .../0_stateless/02371_analyzer_join_cross.sql | 78 + .../02372_analyzer_join_inner.reference | 51 + .../0_stateless/02372_analyzer_join_inner.sql | 156 ++ 34 files changed, 3638 insertions(+), 825 deletions(-) create mode 100644 src/Analyzer/ArrayJoinNode.cpp create mode 100644 src/Analyzer/ArrayJoinNode.h create mode 100644 src/Analyzer/JoinNode.cpp create mode 100644 src/Analyzer/JoinNode.h create mode 100644 tests/queries/0_stateless/02371_analyzer_join_cross.reference create mode 100644 tests/queries/0_stateless/02371_analyzer_join_cross.sql create mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.reference create mode 100644 tests/queries/0_stateless/02372_analyzer_join_inner.sql diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp new file mode 100644 index 00000000000..c85f52b1657 --- /dev/null +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -0,0 +1,76 @@ +#include + +#include +#include +#include + +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +ArrayJoinNode::ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_) + : is_left(is_left_) +{ + children.resize(children_size); + children[table_expression_child_index] = std::move(table_expression_); + children[join_expressions_child_index] = std::move(join_expressions_); +} + +void ArrayJoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "ARRAY_JOIN id: " << format_state.getNodeId(this); + buffer << ", is_left: " << is_left; + + buffer << '\n' << std::string(indent + 2, ' ') << "TABLE EXPRESSION\n"; + getTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + getJoinExpressionsNode()->dumpTreeImpl(buffer, format_state, indent + 4); +} + +bool ArrayJoinNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return is_left == rhs_typed.is_left; +} + +void ArrayJoinNode::updateTreeHashImpl(HashState & state) const +{ + state.update(is_left); +} + +ASTPtr ArrayJoinNode::toASTImpl() const +{ + auto array_join_ast = std::make_shared(); + array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; + const auto & join_expression_list_node = getJoinExpressionsNode(); + array_join_ast->children.push_back(join_expression_list_node->toAST()); + array_join_ast->expression_list = array_join_ast->children.back(); + + ASTPtr tables_in_select_query_ast = std::make_shared(); + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]); + + auto array_join_query_element_ast = std::make_shared(); + array_join_query_element_ast->children.push_back(std::move(array_join_ast)); + array_join_query_element_ast->array_join = array_join_query_element_ast->children.back(); + + tables_in_select_query_ast->children.push_back(std::move(array_join_query_element_ast)); + + return tables_in_select_query_ast; +} + +QueryTreeNodePtr ArrayJoinNode::cloneImpl() const +{ + ArrayJoinNodePtr result_array_join_node(new ArrayJoinNode(is_left)); + return result_array_join_node; +} + +} diff --git a/src/Analyzer/ArrayJoinNode.h b/src/Analyzer/ArrayJoinNode.h new file mode 100644 index 00000000000..248e8443143 --- /dev/null +++ b/src/Analyzer/ArrayJoinNode.h @@ -0,0 +1,105 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include +#include + +namespace DB +{ + +/** Array join node represents array join in query tree. + * + * In query tree join expression is represented by list query tree node. + * + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a, [4, 5, 6] as b. + */ +class ArrayJoinNode; +using ArrayJoinNodePtr = std::shared_ptr; + +class ArrayJoinNode final : public IQueryTreeNode +{ +public: + /** Construct array join node with table expression. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3] as a. + * test_table - table expression. + * join_expression_list - list of array join expressions. + */ + ArrayJoinNode(QueryTreeNodePtr table_expression_, QueryTreeNodePtr join_expressions_, bool is_left_); + + /// Get table expression + const QueryTreeNodePtr & getTableExpression() const + { + return children[table_expression_child_index]; + } + + /// Get table expression + QueryTreeNodePtr & getTableExpression() + { + return children[table_expression_child_index]; + } + + /// Get join expressions + const ListNode & getJoinExpressions() const + { + return children[join_expressions_child_index]->as(); + } + + /// Get join expressions + ListNode & getJoinExpressions() + { + return children[join_expressions_child_index]->as(); + } + + /// Get join expressions node + const QueryTreeNodePtr & getJoinExpressionsNode() const + { + return children[join_expressions_child_index]; + } + + QueryTreeNodePtr & getJoinExpressionsNode() + { + return children[join_expressions_child_index]; + } + + bool isLeft() const + { + return is_left; + } + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::ARRAY_JOIN; + } + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + explicit ArrayJoinNode(bool is_left_) + : is_left(is_left_) + {} + + bool is_left = false; + + static constexpr size_t table_expression_child_index = 0; + static constexpr size_t join_expressions_child_index = 1; + static constexpr size_t children_size = join_expressions_child_index + 1; +}; + +} + diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 28023f7328a..5fdce4c574c 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -20,15 +20,31 @@ ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_sour : column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(1); + children.resize(children_size); } -ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_) +ColumnNode::ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) : column(std::move(column_)) , column_source(std::move(column_source_)) { - children.resize(1); - children[0] = std::move(alias_expression_node_); + children.resize(children_size); + children[expression_child_index] = std::move(expression_node_); +} + +ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_name_qualification(std::move(column_name_qualification_)) + , column_source(std::move(column_source_)) +{ +} + +ColumnNode::ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_) + : column(std::move(column_)) + , column_name_qualification(std::move(column_name_qualification_)) + , column_source(std::move(column_source_)) +{ + children.resize(children_size); + children[expression_child_index] = std::move(expression_node_); } QueryTreeNodePtr ColumnNode::getColumnSource() const @@ -52,23 +68,26 @@ void ColumnNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t buffer << ", column_name: " << column.name << ", result_type: " << column.type->getName(); + if (!column_name_qualification.empty()) + buffer << ", column_name_qualification: " << column_name_qualification; + auto column_source_ptr = column_source.lock(); if (column_source_ptr) buffer << ", source_id: " << state.getNodeId(column_source_ptr.get()); - const auto & alias_expression = getAliasExpression(); + const auto & expression = getExpression(); - if (alias_expression) + if (expression) { - buffer << '\n' << std::string(indent + 2, ' ') << "ALIAS EXPRESSION\n"; - alias_expression->dumpTreeImpl(buffer, state, indent + 4); + buffer << '\n' << std::string(indent + 2, ' ') << "EXPRESSION\n"; + expression->dumpTreeImpl(buffer, state, indent + 4); } } bool ColumnNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - if (column != rhs_typed.column) + if (column != rhs_typed.column || column_name_qualification != rhs_typed.column_name_qualification) return false; auto source_ptr = column_source.lock(); @@ -93,6 +112,9 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const hash_state.update(column_type_name.size()); hash_state.update(column_type_name); + hash_state.update(column_name_qualification.size()); + hash_state.update(column_name_qualification); + auto column_source_ptr = column_source.lock(); if (column_source_ptr) column_source_ptr->updateTreeHashImpl(hash_state); @@ -100,7 +122,7 @@ void ColumnNode::updateTreeHashImpl(HashState & hash_state) const QueryTreeNodePtr ColumnNode::cloneImpl() const { - return std::make_shared(column, column_source); + return std::make_shared(column, column_name_qualification, column_source); } void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) @@ -115,6 +137,12 @@ void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & point ASTPtr ColumnNode::toASTImpl() const { + if (!column_name_qualification.empty()) + { + std::vector parts = {column_name_qualification, column.name}; + return std::make_shared(std::move(parts)); + } + return std::make_shared(column.name); } diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index 5a27e23c5cb..4784064d0d6 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -12,7 +12,14 @@ namespace DB * Column can be table expression, lambda, subquery. * Column source must be valid during column node lifetime. * - * For table ALIAS columns. Column node must contain ALIAS expression. + * For table ALIAS columns. Column node must contain expression. + * For ARRAY JOIN join expression column. Column node must contain expression. + * + * Additionaly column must be initialized with column name qualification if there are multiple + * unqualified columns with same name in query scope. + * Example: SELECT a.id, b.id FROM test_table_join_1 AS a, test_table_join_1. + * Both columns a.id and b.id have same unqualified name id. And additionally must be initialized + * with qualification a and b. * * During query analysis pass identifier node is resolved into column. See IdentifierNode.h. * @@ -34,8 +41,14 @@ public: /// Construct column node with column name, type and column source weak pointer. ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_); - /// Construct ALIAS column node with column name, type, column expression and column source weak pointer. - ColumnNode(NameAndTypePair column_, QueryTreeNodePtr alias_expression_node_, QueryTreeNodeWeakPtr column_source_); + /// Construct expression column node with column name, type, column expression and column source weak pointer. + ColumnNode(NameAndTypePair column_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); + + /// Construct column node with column name, type, column name qualification and column source weak pointer. + ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodeWeakPtr column_source_); + + /// Construct expression column node with column name, type, column name qualification column expression and column source weak pointer. + ColumnNode(NameAndTypePair column_, String column_name_qualification_, QueryTreeNodePtr expression_node_, QueryTreeNodeWeakPtr column_source_); /// Get column const NameAndTypePair & getColumn() const @@ -49,25 +62,35 @@ public: return column.name; } + bool hasColumnNameQualfication() const + { + return !column_name_qualification.empty(); + } + + const String & getColumnQualification() const + { + return column_name_qualification; + } + /// Get column type const DataTypePtr & getColumnType() const { return column.type; } - bool hasAliasExpression() const + bool hasExpression() const { - return children[alias_expression_child_index] != nullptr; + return children[expression_child_index] != nullptr; } - const QueryTreeNodePtr & getAliasExpression() const + const QueryTreeNodePtr & getExpression() const { - return children[alias_expression_child_index]; + return children[expression_child_index]; } - QueryTreeNodePtr & getAliasExpression() + QueryTreeNodePtr & getExpression() { - return children[alias_expression_child_index]; + return children[expression_child_index]; } /** Get column source. @@ -111,9 +134,11 @@ protected: private: NameAndTypePair column; + String column_name_qualification; QueryTreeNodeWeakPtr column_source; - static constexpr size_t alias_expression_child_index = 0; + static constexpr size_t expression_child_index = 0; + static constexpr size_t children_size = expression_child_index + 1; }; } diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index b947d43ec0f..051976db228 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -33,6 +33,8 @@ const char * toString(QueryTreeNodeType type) case QueryTreeNodeType::TABLE: return "TABLE"; case QueryTreeNodeType::TABLE_FUNCTION: return "TABLE_FUNCTION"; case QueryTreeNodeType::QUERY: return "QUERY"; + case QueryTreeNodeType::ARRAY_JOIN: return "ARRAY_JOIN"; + case QueryTreeNodeType::JOIN: return "JOIN"; } } diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 75c3af87c3a..b45ec003bdc 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -38,6 +38,8 @@ enum class QueryTreeNodeType TABLE, TABLE_FUNCTION, QUERY, + ARRAY_JOIN, + JOIN }; /// Convert query tree node type to string diff --git a/src/Analyzer/JoinNode.cpp b/src/Analyzer/JoinNode.cpp new file mode 100644 index 00000000000..5227b82c119 --- /dev/null +++ b/src/Analyzer/JoinNode.cpp @@ -0,0 +1,159 @@ +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodePtr join_expression_, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{ + children.resize(children_size); + children[left_table_expression_child_index] = std::move(left_table_expression_); + children[right_table_expression_child_index] = std::move(right_table_expression_); + children[join_expression_child_index] = std::move(join_expression_); +} + +JoinNode::JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodes using_identifiers, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{ + children.resize(children_size); + children[left_table_expression_child_index] = std::move(left_table_expression_); + children[right_table_expression_child_index] = std::move(right_table_expression_); + children[join_expression_child_index] = std::make_shared(std::move(using_identifiers)); +} + +JoinNode::JoinNode(JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_) + : locality(locality_) + , strictness(strictness_) + , kind(kind_) +{} + +void JoinNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const +{ + buffer << std::string(indent, ' ') << "JOIN id: " << format_state.getNodeId(this); + + if (locality != JoinLocality::Unspecified) + buffer << ", locality: " << toString(locality); + + if (strictness != JoinStrictness::Unspecified) + buffer << ", strictness: " << toString(strictness); + + buffer << ", kind: " << toString(kind); + + buffer << '\n' << std::string(indent + 2, ' ') << "LEFT TABLE EXPRESSION\n"; + getLeftTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + buffer << '\n' << std::string(indent + 2, ' ') << "RIGHT TABLE EXPRESSSION\n"; + getRightTableExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + + if (getJoinExpression()) + { + buffer << '\n' << std::string(indent + 2, ' ') << "JOIN EXPRESSSION\n"; + getJoinExpression()->dumpTreeImpl(buffer, format_state, indent + 4); + } +} + +bool JoinNode::isEqualImpl(const IQueryTreeNode & rhs) const +{ + const auto & rhs_typed = assert_cast(rhs); + return locality == rhs_typed.locality && strictness == rhs_typed.strictness && kind == rhs_typed.kind; +} + +void JoinNode::updateTreeHashImpl(HashState & state) const +{ + state.update(locality); + state.update(strictness); + state.update(kind); +} + +ASTPtr JoinNode::toASTTableJoin() const +{ + auto join_ast = std::make_shared(); + join_ast->locality = locality; + join_ast->strictness = strictness; + join_ast->kind = kind; + + if (children[join_expression_child_index]) + { + auto join_expression_ast = children[join_expression_child_index]->toAST(); + + if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) + join_ast->using_expression_list = std::move(join_expression_ast); + else + join_ast->on_expression = std::move(join_expression_ast); + } + + return join_ast; +} + +ASTPtr JoinNode::toASTImpl() const +{ + ASTPtr tables_in_select_query_ast = std::make_shared(); + + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[left_table_expression_child_index]); + + size_t join_table_index = tables_in_select_query_ast->children.size(); + + auto join_ast = std::make_shared(); + join_ast->locality = locality; + join_ast->strictness = strictness; + join_ast->kind = kind; + + if (children[join_expression_child_index]) + { + auto join_expression_ast = children[join_expression_child_index]->toAST(); + + if (children[join_expression_child_index]->getNodeType() == QueryTreeNodeType::LIST) + join_ast->using_expression_list = std::move(join_expression_ast); + else + join_ast->on_expression = std::move(join_expression_ast); + } + + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[right_table_expression_child_index]); + + auto & table_element = tables_in_select_query_ast->children.at(join_table_index)->as(); + table_element.children.push_back(std::move(join_ast)); + table_element.table_join = table_element.children.back(); + + return tables_in_select_query_ast; +} + +QueryTreeNodePtr JoinNode::cloneImpl() const +{ + JoinNodePtr result_join_node(new JoinNode(locality, strictness, kind)); + return result_join_node; +} + +} diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h new file mode 100644 index 00000000000..3bde9a25b84 --- /dev/null +++ b/src/Analyzer/JoinNode.h @@ -0,0 +1,148 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include + +#include + +namespace DB +{ + +/** Array join node represents array join in query tree. + * Example: SELECT id FROM test_table ARRAY JOIN [1, 2, 3]. + */ +class JoinNode; +using JoinNodePtr = std::shared_ptr; + +class JoinNode final : public IQueryTreeNode +{ +public: + /** Construct join node with left table expression, right table expression and join expression. + * Example: SELECT id FROM test_table_1 INNER JOIN test_table_2 ON expression. + * + * test_table_1 - left table expression. + * test_table_2 - right table expression. + * join_expression - join_expression; + */ + JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodePtr join_expression_, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + /** Construct join node with left table expression, right table expression and using identifiers. + * Example: SELECT id FROM test_table_1 INNER JOIN test_table_2 USING (using_identifier, ...). + * test_table_1 - left table expression. + * test_table_2 - right table expression. + * (using_identifier, ...) - using identifiers. + */ + JoinNode(QueryTreeNodePtr left_table_expression_, + QueryTreeNodePtr right_table_expression_, + QueryTreeNodes using_identifiers, + JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + /// Get left table expression + const QueryTreeNodePtr & getLeftTableExpression() const + { + return children[left_table_expression_child_index]; + } + + /// Get left table expression + QueryTreeNodePtr & getLeftTableExpression() + { + return children[left_table_expression_child_index]; + } + + /// Get right table expression + const QueryTreeNodePtr & getRightTableExpression() const + { + return children[right_table_expression_child_index]; + } + + /// Get right table expression + QueryTreeNodePtr & getRightTableExpression() + { + return children[right_table_expression_child_index]; + } + + /// Get join expression + const QueryTreeNodePtr & getJoinExpression() const + { + return children[join_expression_child_index]; + } + + /// Get join expression + QueryTreeNodePtr & getJoinExpression() + { + return children[join_expression_child_index]; + } + + bool isUsingJoinExpression() const + { + return getJoinExpression() && getJoinExpression()->getNodeType() == QueryTreeNodeType::LIST; + } + + bool isOnJoinExpression() const + { + return getJoinExpression() && getJoinExpression()->getNodeType() != QueryTreeNodeType::LIST; + } + + JoinLocality getLocality() const + { + return locality; + } + + JoinStrictness getStrictness() const + { + return strictness; + } + + JoinKind getKind() const + { + return kind; + } + + ASTPtr toASTTableJoin() const; + + QueryTreeNodeType getNodeType() const override + { + return QueryTreeNodeType::JOIN; + } + + void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; + + bool isEqualImpl(const IQueryTreeNode & rhs) const override; + + void updateTreeHashImpl(HashState & state) const override; + +protected: + ASTPtr toASTImpl() const override; + + QueryTreeNodePtr cloneImpl() const override; + +private: + JoinNode(JoinLocality locality_, + JoinStrictness strictness_, + JoinKind kind_); + + JoinLocality locality = JoinLocality::Unspecified; + JoinStrictness strictness = JoinStrictness::Unspecified; + JoinKind kind = JoinKind::Inner; + + static constexpr size_t left_table_expression_child_index = 0; + static constexpr size_t right_table_expression_child_index = 1; + static constexpr size_t join_expression_child_index = 2; + static constexpr size_t children_size = join_expression_child_index + 1; +}; + +} + diff --git a/src/Analyzer/ListNode.cpp b/src/Analyzer/ListNode.cpp index ffd72a3990e..59b42d384de 100644 --- a/src/Analyzer/ListNode.cpp +++ b/src/Analyzer/ListNode.cpp @@ -11,6 +11,11 @@ namespace DB { +ListNode::ListNode(QueryTreeNodes nodes) +{ + children = std::move(nodes); +} + void ListNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "LIST id: " << format_state.getNodeId(this); diff --git a/src/Analyzer/ListNode.h b/src/Analyzer/ListNode.h index 413dc964612..f93043144e0 100644 --- a/src/Analyzer/ListNode.h +++ b/src/Analyzer/ListNode.h @@ -16,6 +16,12 @@ using ListNodePtr = std::shared_ptr; class ListNode final : public IQueryTreeNode { public: + /// Initialize list node with empty nodes + ListNode() = default; + + /// Initialize list node with nodes + explicit ListNode(QueryTreeNodes nodes); + /// Get list nodes const QueryTreeNodes & getNodes() const { diff --git a/src/Analyzer/MatcherNode.cpp b/src/Analyzer/MatcherNode.cpp index 3aa70557987..6393193ec01 100644 --- a/src/Analyzer/MatcherNode.cpp +++ b/src/Analyzer/MatcherNode.cpp @@ -178,6 +178,8 @@ String MatcherNode::getName() const } buffer << ')'; + /// TODO: Transformers + return buffer.str(); } diff --git a/src/Analyzer/QueryAnalysisPass.cpp b/src/Analyzer/QueryAnalysisPass.cpp index d20e8598e03..b31457f57c0 100644 --- a/src/Analyzer/QueryAnalysisPass.cpp +++ b/src/Analyzer/QueryAnalysisPass.cpp @@ -42,6 +42,8 @@ #include #include #include +#include +#include #include #include @@ -82,6 +84,7 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_TABLE; extern const int INCORRECT_ELEMENT_OF_SET; + extern const int TYPE_MISMATCH; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h before. @@ -129,7 +132,7 @@ namespace ErrorCodes * Resolution starts with current scope. * 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority. * 2. Try to resolve identifier from aliases. - * 3. Try to resolve identifier from tables if scope is query, or if there are registered table columns in scope. + * 3. Try to resolve identifier from join tree if scope is query, or if there are registered table columns in scope. * Steps 2 and 3 can be changed using prefer_column_name_to_alias setting. * 4. If it is table lookup, try to resolve identifier from CTE. * If identifier could not be resolved in current scope, resolution must be continued in parent scopes. @@ -151,7 +154,7 @@ namespace ErrorCodes * CREATE TABLE test_table (id UInt64, compound_value Tuple(value UInt64)) ENGINE=TinyLog; * SELECT compound_value.value, 1 AS compound_value FROM test_table; * Identifier first part compound_value bound to entity with alias compound_value, but nested identifier part cannot be resolved from entity, - * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from tables. + * lookup should not be continued, and exception must be throwed because if lookup continues that way identifier can be resolved from join tree. * * TODO: This was not supported properly before analyzer because nested identifier could not be resolved from alias. * @@ -162,16 +165,23 @@ namespace ErrorCodes * lookup should not be continued, and exception must be throwed because if lookup continues identifier can be resolved from parent scope. * * TODO: Add expression name into query tree node. Example: SELECT plus(1, 1). Result: SELECT 2. Expression name of constant node should be 2. + * TODO: Disable identifier cache during join tree resolve * TODO: Update exception messages + * TODO: JOIN TREE subquery constant columns + * TODO: Column name qualifications * TODO: Table identifiers with optional UUID. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: UNION - * TODO: JOIN, ARRAY JOIN + * TODO: JOIN support LEFT, RIGHT with nullable. JOIN support USING. JOIN support SELF JOIN with MergeTree. JOIN support matchers. * TODO: WINDOW functions + * TODO: Table expression modifiers final, sample_size, sample_offset */ +namespace +{ + /// Identifier lookup context enum class IdentifierLookupContext : uint8_t { @@ -234,7 +244,7 @@ inline bool operator==(const IdentifierLookup & lhs, const IdentifierLookup & rh return lhs.identifier.getFullName() == rhs.identifier.getFullName() && lhs.lookup_context == rhs.lookup_context; } -inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs) +[[maybe_unused]] inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs) { return !(lhs == rhs); } @@ -252,7 +262,7 @@ enum class IdentifierResolvePlace : UInt8 NONE = 0, EXPRESSION_ARGUMENTS, ALIASES, - TABLES, + JOIN_TREE, /// Valid only for table lookup CTE, /// Valid only for table lookup @@ -266,7 +276,7 @@ static const char * toString(IdentifierResolvePlace resolved_identifier_place) case IdentifierResolvePlace::NONE: return "NONE"; case IdentifierResolvePlace::EXPRESSION_ARGUMENTS: return "EXPRESSION_ARGUMENTS"; case IdentifierResolvePlace::ALIASES: return "ALIASES"; - case IdentifierResolvePlace::TABLES: return "TABLES"; + case IdentifierResolvePlace::JOIN_TREE: return "JOIN_TREE"; case IdentifierResolvePlace::CTE: return "CTE"; case IdentifierResolvePlace::DATABASE_CATALOG: return "DATABASE_CATALOG"; } @@ -280,32 +290,32 @@ struct IdentifierResolveResult IdentifierResolvePlace resolve_place = IdentifierResolvePlace::NONE; bool resolved_from_parent_scopes = false; - bool isResolved() const + [[maybe_unused]] bool isResolved() const { return resolve_place != IdentifierResolvePlace::NONE; } - bool isResolvedFromParentScopes() const + [[maybe_unused]] bool isResolvedFromParentScopes() const { return resolved_from_parent_scopes; } - bool isResolvedFromExpressionArguments() const + [[maybe_unused]] bool isResolvedFromExpressionArguments() const { return resolve_place == IdentifierResolvePlace::EXPRESSION_ARGUMENTS; } - bool isResolvedFromAliases() const + [[maybe_unused]] bool isResolvedFromAliases() const { return resolve_place == IdentifierResolvePlace::ALIASES; } - bool isResolvedFromTables() const + [[maybe_unused]] bool isResolvedFromJoinTree() const { - return resolve_place == IdentifierResolvePlace::TABLES; + return resolve_place == IdentifierResolvePlace::JOIN_TREE; } - bool isResolvedFromCTEs() const + [[maybe_unused]] bool isResolvedFromCTEs() const { return resolve_place == IdentifierResolvePlace::CTE; } @@ -320,6 +330,14 @@ struct IdentifierResolveResult buffer << resolved_identifier->formatASTForErrorMessage() << " place " << toString(resolve_place) << " resolved from parent scopes " << resolved_from_parent_scopes; } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } }; struct IdentifierResolveSettings @@ -327,6 +345,9 @@ struct IdentifierResolveSettings /// Allow to check parent scopes during identifier resolution bool allow_to_check_parent_scopes = true; + /// Allow to check join tree during identifier resolution + bool allow_to_check_join_tree = true; + /// Allow to check CTEs during table identifer resolution bool allow_to_check_cte = true; @@ -344,14 +365,14 @@ struct TableExpressionColumns return column_identifier_first_parts.find(std::string(identifier.at(0))) != column_identifier_first_parts.end(); } - void dump(WriteBuffer & buffer) + [[maybe_unused]] void dump(WriteBuffer & buffer) { buffer << "Columns size " << column_name_to_column_node.size() << '\n'; for (auto & [column_name, column_node] : column_name_to_column_node) { buffer << "Column name " << column_name << " column node " << column_node->formatASTForErrorMessage(); - buffer << " is alias " << column_node->hasAliasExpression() << '\n'; + buffer << " is alias " << column_node->hasExpression() << '\n'; } } }; @@ -407,7 +428,7 @@ public: return alias_name_to_expressions_size.find(alias) != alias_name_to_expressions_size.end(); } - size_t size() const + [[maybe_unused]] size_t size() const { return expressions.size(); } @@ -475,6 +496,9 @@ struct IdentifierResolveScope /// Current scope expression in resolve process stack ExpressionsStack expressions_in_resolve_process_stack; + /// Table expressions in resolve process + std::unordered_set table_expressions_in_resolve_process; + /// Current scope expression std::unordered_set non_cached_identifier_lookups_during_expression_resolve; @@ -482,7 +506,7 @@ struct IdentifierResolveScope bool allow_to_check_parent_scopes = true; /// Dump identifier resolve scope - void dump(WriteBuffer & buffer) + [[maybe_unused]] void dump(WriteBuffer & buffer) { buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; buffer << "Identifier lookup to result " << identifier_lookup_to_result.size() << '\n'; @@ -490,6 +514,7 @@ struct IdentifierResolveScope { buffer << "Identifier " << identifier.dump() << " resolve result "; result.dump(buffer); + buffer << '\n'; } buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n'; @@ -524,6 +549,162 @@ struct IdentifierResolveScope } }; + +/** Visitor that extracts expression and function aliases from node and initialize scope tables with it. + * Does not go into child lambdas and queries. + * + * Important: + * Identifier nodes with aliases are added both in alias to expression and alias to function map. + * + * These is necessary because identifier with alias can give alias name to any query tree node. + * + * Example: + * WITH (x -> x + 1) AS id, id AS value SELECT value(1); + * In this example id as value is identifier node that has alias, during scope initialization we cannot derive + * that id is actually lambda or expression. + * + * There are no easy solution here, without trying to make full featured expression resolution at this stage. + * Example: + * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); + * + * It is client responsibility after resolving identifier node with alias, make following actions: + * 1. If identifier node was resolved in function scope, remove alias from scope expression map. + * 2. If identifier node was resolved in expression scope, remove alias from scope function map. + * + * That way we separate alias map initialization and expressions resolution. + */ +class QueryExpressionsAliasVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + IdentifierResolveScope & scope; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + updateAliasesIfNeeded(data, node, false); + } + + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) + { + if (auto * lambda_node = child->as()) + { + updateAliasesIfNeeded(data, child, true); + return false; + } + else if (auto * query_tree_node = child->as()) + { + if (query_tree_node->isCTE()) + return false; + + updateAliasesIfNeeded(data, child, false); + return false; + } + + return true; + } +private: + static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) + { + if (!node->hasAlias()) + return; + + const auto & alias = node->getAlias(); + + if (function_node) + { + if (data.scope.alias_name_to_expression_node.contains(alias)) + data.scope.nodes_with_duplicated_aliases.insert(node); + + auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + if (!inserted) + data.scope.nodes_with_duplicated_aliases.insert(node); + + return; + } + + if (data.scope.alias_name_to_lambda_node.contains(alias)) + data.scope.nodes_with_duplicated_aliases.insert(node); + + auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); + if (!inserted) + data.scope.nodes_with_duplicated_aliases.insert(node); + + /// If node is identifier put it also in scope alias name to lambda node map + if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) + data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); + } +}; + +using QueryExpressionsAliasVisitor = QueryExpressionsAliasVisitorMatcher::Visitor; + +class TableExpressionsAliasVisitorMatcher +{ +public: + using Visitor = InDepthQueryTreeVisitor; + + struct Data + { + IdentifierResolveScope & scope; + }; + + static void visit(QueryTreeNodePtr & node, Data & data) + { + updateAliasesIfNeeded(data, node); + } + + static bool needChildVisit(const QueryTreeNodePtr & node, const QueryTreeNodePtr & child) + { + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::ARRAY_JOIN: + { + const auto & array_join_node = node->as(); + return child.get() == array_join_node.getTableExpression().get(); + } + case QueryTreeNodeType::JOIN: + { + const auto & join_node = node->as(); + return child.get() == join_node.getLeftTableExpression().get() || child.get() == join_node.getRightTableExpression().get(); + } + default: + { + break; + } + } + + return false; + } + +private: + static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node) + { + if (!node->hasAlias()) + return; + + const auto & node_alias = node->getAlias(); + auto [_, inserted] = data.scope.alias_name_to_table_expression_node.emplace(node_alias, node); + if (!inserted) + throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "Multiple table expressions with same alias {}. In scope {}", + node_alias, + data.scope.scope_node->formatASTForErrorMessage()); + } +}; + +using TableExpressionsAliasVisitor = TableExpressionsAliasVisitorMatcher::Visitor; + +struct StorageLockAndSnapshot +{ + TableLockHolder lock; + StorageSnapshotPtr snapshot; +}; + class QueryAnalyzer { public: @@ -565,8 +746,6 @@ private: static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); - static NamesAndTypesList getQueryNodeColumns(const QueryTreeNodePtr & query_node); - QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunction(const std::string & function_name); void evaluateScalarSubquery(QueryTreeNodePtr & query_tree_node); @@ -581,7 +760,11 @@ private: QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); - QueryTreeNodePtr tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); + + QueryTreeNodePtr tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); IdentifierResolveResult tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -599,9 +782,11 @@ private: void resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression); - void initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope); - void resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope); + void initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + + void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor); void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope); @@ -611,11 +796,11 @@ private: /// Lambdas that are currently in resolve process std::unordered_set lambdas_in_resolve_process; - /// Query nodes where FROM section is in resolve process - std::unordered_set query_node_from_section_in_resolve_process; - /// Table expression node to columns cache std::unordered_map table_expression_node_to_columns; + + /// Array join expressions counter + size_t array_join_expressions_counter = 0; }; /// Utility functions implementation @@ -639,33 +824,6 @@ QueryTreeNodePtr QueryAnalyzer::wrapExpressionNodeInTupleElement(QueryTreeNodePt return expression_node; } -/** Get query node columns using projection section - */ -NamesAndTypesList QueryAnalyzer::getQueryNodeColumns(const QueryTreeNodePtr & query_node) -{ - const auto & query_node_typed = query_node->as(); - - NamesAndTypes query_columns; - - const auto & projection_nodes = query_node_typed.getProjection(); - query_columns.reserve(projection_nodes.getNodes().size()); - - for (const auto & projection_node : projection_nodes.getNodes()) - { - auto column_type = projection_node->getResultType(); - std::string column_name; - - if (projection_node->hasAlias()) - column_name = projection_node->getAlias(); - else - column_name = projection_node->getName(); - - query_columns.emplace_back(column_name, column_type); - } - - return {query_columns.begin(), query_columns.end()}; -} - /** Try to get lambda node from sql user defined functions if sql user defined function with function name exists. * Returns lambda node if function exists, nullptr otherwise. */ @@ -818,12 +976,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con if (!database) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name); - auto table = database->tryGetTable(table_name, context); + auto storage = database->tryGetTable(table_name, context); - if (!table) + if (!storage) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name); - return std::make_shared(std::move(table), context); + auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); + auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); + return std::make_shared(std::move(storage), storage_lock, storage_snapshot); } /** Resolve identifier from expression arguments. @@ -875,97 +1035,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons return it->second; } -/** Visitor that extracts expression and function aliases from node and initialize scope tables with it. - * Does not go into child lambdas and queries. - * - * Important: - * Identifier nodes with aliases are added both in alias to expression and alias to function map. - * - * These is necessary because identifier with alias can give alias name to any query tree node. - * - * Example: - * WITH (x -> x + 1) AS id, id AS value SELECT value(1); - * In this example id as value is identifier node that has alias, during scope initialization we cannot derive - * that id is actually lambda or expression. - * - * There are no easy solution here, without trying to make full featured expression resolution at this stage. - * Example: - * WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1); - * - * It is client responsibility after resolving identifier node with alias, make following actions: - * 1. If identifier node was resolved in function scope, remove alias from scope expression map. - * 2. If identifier node was resolved in expression scope, remove alias from scope function map. - * - * That way we separate alias map initialization and expressions resolution. - */ -class ScopeAliasVisitorMatcher -{ -public: - using Visitor = InDepthQueryTreeVisitor; - - struct Data - { - IdentifierResolveScope & scope; - }; - - static void visit(QueryTreeNodePtr & node, Data & data) - { - updateAliasesIfNeeded(data, node, false); - } - - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child, Data & data) - { - if (auto * lambda_node = child->as()) - { - updateAliasesIfNeeded(data, child, true); - return false; - } - else if (auto * query_tree_node = child->as()) - { - if (query_tree_node->isCTE()) - return false; - - updateAliasesIfNeeded(data, child, false); - return false; - } - - return !(child->as()); - } -private: - static void updateAliasesIfNeeded(Data & data, const QueryTreeNodePtr & node, bool function_node) - { - if (!node->hasAlias()) - return; - - const auto & alias = node->getAlias(); - - if (function_node) - { - if (data.scope.alias_name_to_expression_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); - - auto [_, inserted] = data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); - if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); - - return; - } - - if (data.scope.alias_name_to_lambda_node.contains(alias)) - data.scope.nodes_with_duplicated_aliases.insert(node); - - auto [_, inserted] = data.scope.alias_name_to_expression_node.insert(std::make_pair(alias, node)); - if (!inserted) - data.scope.nodes_with_duplicated_aliases.insert(node); - - /// If node is identifier put it also in scope alias name to lambda node map - if (node->getNodeType() == QueryTreeNodeType::IDENTIFIER) - data.scope.alias_name_to_lambda_node.insert(std::make_pair(alias, node)); - } -}; - -using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; - /** Resolve identifier from scope aliases. * * Resolve strategy: @@ -1000,7 +1069,7 @@ using ScopeAliasVisitor = ScopeAliasVisitorMatcher::Visitor; * Example: SELECT value.a, cast('(1)', 'Tuple(a UInt64)') AS value; * * Special case for IdentifierNode, if node is identifier depending on lookup context we need to erase entry from expression or lambda map. - * Check ScopeAliasVisitorMatcher documentation. + * Check QueryExpressionsAliasVisitor documentation. * * Special case for QueryNode, if lookup context is expression, evaluate it as scalar subquery. * @@ -1062,22 +1131,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto lookup_result = tryResolveIdentifier(IdentifierLookup{alias_identifier->getIdentifier(), identifier_lookup.lookup_context}, scope, identifier_resolve_settings); it->second = lookup_result.resolved_identifier; - if (!it->second) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {} identifier {}. In scope {}", - toStringLowercase(identifier_lookup.lookup_context), - identifier_lookup.identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); - /** During collection of aliases if node is identifier and has alias, we cannot say if it is - * column or function node. Check ScopeAliasVisitor documentation for clarification. + * column or function node. Check QueryExpressionsAliasVisitor documentation for clarification. * * If we resolved identifier node as expression, we must remove identifier node alias from * function alias map. * If we resolved identifier node as function, we must remove identifier node alias from * expression alias map. */ - if (identifier_lookup.isExpressionLookup()) + if (identifier_lookup.isExpressionLookup() && it->second) scope.alias_name_to_lambda_node.erase(identifier_bind_part); else if (identifier_lookup.isFunctionLookup() && it->second) scope.alias_name_to_expression_node.erase(identifier_bind_part); @@ -1141,8 +1203,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident return {}; } - if (it->second->hasAliasExpression()) - resolveExpressionNode(it->second->getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (it->second->hasExpression()) + resolveExpressionNode(it->second->getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); QueryTreeNodePtr result = it->second; @@ -1160,47 +1222,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident return result; } -/** Resolve identifier from scope tables. - * - * 1. If identifier is in function lookup context return nullptr. - * 2. Try to resolve identifier from table columns. - * 3. If there is no FROM section return nullptr. - * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. - * If identifer has 2 parts try to match it with database_name and table_name. - * If identifier has 1 part try to match it with table_name, then try to match it with table alias. - * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. - * Start with identifier first part, if it match some column name in table try to get column with full identifier name. - * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. - */ -QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTable(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (identifier_lookup.isFunctionLookup()) - return {}; - - /// Try to resolve identifier from table columns - - if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) - return resolved_identifier; - - auto * query_scope_node = scope.scope_node->as(); - if (!query_scope_node || !query_scope_node->getFrom()) - return {}; - - auto from_node = query_scope_node->getFrom(); - - auto * table_node = from_node->as(); - auto * query_node = from_node->as(); - auto * table_function_node = from_node->as(); - - /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. - * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; - * TODO: This can be supported - */ - if (query_node_from_section_in_resolve_process.contains(query_scope_node)) - return {}; + auto * table_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); if (!table_node && !table_function_node && !query_node) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "FROM does not contain table, table function or query node. Actual {}", from_node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); const auto & identifier = identifier_lookup.identifier; const auto & path_start = identifier.getParts().front(); @@ -1240,102 +1272,25 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL table_node->formatASTForErrorMessage()); if (parts_size == 1 && path_start == table_name) - return from_node; + return table_expression_node; else if (parts_size == 2 && path_start == database_name && identifier[1] == table_name) - return from_node; + return table_expression_node; else return {}; } - auto storage_columns_it = table_expression_node_to_columns.find(from_node.get()); + auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); if (storage_columns_it == table_expression_node_to_columns.end()) - { - TableExpressionColumns storage_columns; - - if (table_node || table_function_node) - { - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); - - auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); - const auto & columns_description = storage_snapshot->metadata->getColumns(); - - std::vector> alias_columns_to_resolve; - std::unordered_map column_name_to_column_node; - column_name_to_column_node.reserve(column_names_and_types.size()); - - for (const auto & column_name_and_type : column_names_and_types) - { - const auto & column_default = columns_description.getDefault(column_name_and_type.name); - - if (column_default && column_default->kind == ColumnDefaultKind::Alias) - { - auto argument_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), from_node); - column_name_to_column_node.emplace(column_name_and_type.name, argument_node); - alias_columns_to_resolve.emplace_back(column_name_and_type.name, argument_node); - } - else - { - auto argument_node = std::make_shared(column_name_and_type, from_node); - column_name_to_column_node.emplace(column_name_and_type.name, argument_node); - } - } - - for (auto & [alias_column_to_resolve_name, alias_column_to_resolve] : alias_columns_to_resolve) - { - /** Alias column could be potentially resolved during resolve of other ALIAS column. - * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + alias_value_2, alias_value_2 ALIAS id + 5) ENGINE=TinyLog; - * - * During resolve of alias_value_1, alias_value_2 column will be resolved. - */ - alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; - - IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); - alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); - - /// Initialize aliases in alias column scope - ScopeAliasVisitorMatcher::Data data{alias_column_resolve_scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); - - visitor.visit(alias_column_to_resolve->getAliasExpression()); - - resolveExpressionNode(alias_column_resolve_scope.scope_node, alias_column_resolve_scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - - column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); - column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; - } - - storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); - } - else if (query_node) - { - auto column_names_and_types = getQueryNodeColumns(from_node); - storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); - - for (const auto & column_name_and_type : column_names_and_types) - { - auto column_node = std::make_shared(column_name_and_type, from_node); - storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); - } - } - - storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); - - for (auto & [column_name, _] : storage_columns.column_name_to_column_node) - { - Identifier column_name_identifier(column_name); - storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); - } - - auto [inserted_it, _] = table_expression_node_to_columns.emplace(from_node.get(), std::move(storage_columns)); - storage_columns_it = inserted_it; - } + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table expression {} columns must be initialized. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); auto & storage_columns = storage_columns_it->second; - - auto resolve_identifier_from_storage_or_throw = [&](size_t drop_first_parts_size) -> QueryTreeNodePtr + auto resolve_identifier_from_storage_or_throw = [&](size_t identifier_column_qualifier_parts) -> QueryTreeNodePtr { auto identifier_view = IdentifierView(identifier); - identifier_view.popFirst(drop_first_parts_size); + identifier_view.popFirst(identifier_column_qualifier_parts); /** Compound identifier cannot be resolved directly from storage if storage is not table. * @@ -1346,7 +1301,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * Here there is no column with name test_subquery.compound_expression.value, and additional wrap in tuple element is required. */ - QueryTreeNodePtr result_column; + ColumnNodePtr result_column; bool compound_identifier = identifier_view.getPartsSize() > 1; bool match_full_identifier = false; @@ -1364,7 +1319,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL result_column = it->second; } - if (!result_column) + if (!result_column || (!match_full_identifier && !compound_identifier)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Identifier {} cannot be resolved from {}{}. In scope {}", identifier.getFullName(), @@ -1372,6 +1327,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL table_expression_name.empty() ? "" : " with name " + table_expression_name, scope.scope_node->formatASTForErrorMessage()); + if (identifier_column_qualifier_parts > 0) + { + auto identifier_column_qualifier_view = IdentifierView(identifier); + identifier_column_qualifier_view.popLast(identifier.getPartsSize() - identifier_column_qualifier_parts); + + result_column = std::make_shared(result_column->getColumn(), + std::string(identifier_column_qualifier_view.getFullName()), + result_column->getExpression(), + table_expression_node); + } + QueryTreeNodePtr result_expression = result_column; if (!match_full_identifier && compound_identifier) @@ -1392,23 +1358,137 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTables(const IdentifierL * 3. Try to bind identifier first parts to database name and table name, if true remove first two parts and try to get full identifier from table or throw exception. */ if (storage_columns.canBindIdentifier(IdentifierView(identifier))) - return resolve_identifier_from_storage_or_throw(0 /*drop_first_parts_size*/); + return resolve_identifier_from_storage_or_throw(0 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 1) return {}; - if ((!table_name.empty() && path_start == table_name) || (from_node->hasAlias() && path_start == from_node->getAlias())) - return resolve_identifier_from_storage_or_throw(1 /*drop_first_parts_size*/); + if ((!table_name.empty() && path_start == table_name) || (table_expression_node->hasAlias() && path_start == table_expression_node->getAlias())) + return resolve_identifier_from_storage_or_throw(1 /*identifier_column_qualifier_parts*/); if (identifier.getPartsSize() == 2) return {}; if (!database_name.empty() && path_start == database_name && identifier[1] == table_name) - return resolve_identifier_from_storage_or_throw(2 /*drop_first_parts_size*/); + return resolve_identifier_from_storage_or_throw(2 /*identifier_column_qualifier_parts*/); return {}; } +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::JOIN: + { + const auto & from_join_node = join_tree_node->as(); + + auto left_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getLeftTableExpression(), scope); + auto right_resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_join_node.getRightTableExpression(), scope); + + if (left_resolved_identifier && right_resolved_identifier) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "JOIN {} ambigious identifier {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + identifier_lookup.identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + else if (left_resolved_identifier) + return left_resolved_identifier; + else if (right_resolved_identifier) + return right_resolved_identifier; + + return {}; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + const auto & from_array_join_node = join_tree_node->as(); + auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); + + /** Special case when qualified or unqualified identifier point to array join expression. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; + * + * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. + */ + if (resolved_identifier) + { + for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) + { + auto * array_join_column_expression = array_join_expression->as(); + if (!array_join_column_expression) + continue; + + auto & array_join_column_inner_expression = array_join_column_expression->getExpression(); + + if (!array_join_column_inner_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "ARRAY JOIN column node must have inner expression. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + + if (array_join_expression.get() == resolved_identifier.get() || array_join_expression->isEqual(*resolved_identifier)) + return array_join_expression; + } + } + + return resolved_identifier; + } + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + /** Edge case scenario when subquery in FROM node try to resolve identifier from parent scopes, when FROM is not resolved. + * SELECT subquery.b AS value FROM (SELECT value, 1 AS b) AS subquery; + * TODO: This can be supported + */ + if (scope.table_expressions_in_resolve_process.contains(join_tree_node.get())) + return {}; + + return tryResolveIdentifierFromTable(identifier_lookup, join_tree_node, scope); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Scope FROM section expected table, table function, query, join or array join. Actual {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } +} + +/** Resolve identifier from scope join tree. + * + * 1. If identifier is in function lookup context return nullptr. + * 2. Try to resolve identifier from table columns. + * 3. If there is no FROM section return nullptr. + * 4. If identifier is in table lookup context, check if it has 1 or 2 parts, otherwise throw exception. + * If identifer has 2 parts try to match it with database_name and table_name. + * If identifier has 1 part try to match it with table_name, then try to match it with table alias. + * 5. If identifier is in expression lookup context, we first need to bind identifier to some table column using identifier first part. + * Start with identifier first part, if it match some column name in table try to get column with full identifier name. + * TODO: Need to check if it is okay to throw exception if compound identifier first part bind to column but column is not valid. + */ +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) +{ + if (identifier_lookup.isFunctionLookup()) + return {}; + + /// Try to resolve identifier from table columns + if (auto resolved_identifier = tryResolveIdentifierFromTableColumns(identifier_lookup, scope)) + return resolved_identifier; + + auto * query_scope_node = scope.scope_node->as(); + if (!query_scope_node || !query_scope_node->getFrom()) + return {}; + + const auto & join_tree_node = query_scope_node->getFrom(); + return tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_tree_node, scope); +} + /** Try resolve identifier in current scope parent scopes. * * TODO: If column is matched, throw exception that nested subqueries are not supported. @@ -1495,11 +1575,11 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const * 3. Try resolve identifier from expression arguments. * * If prefer_column_name_to_alias = true. - * 4. Try to resolve identifier from tables. + * 4. Try to resolve identifier from join tree. * 5. Try to resolve identifier from aliases. * Otherwise. * 4. Try to resolve identifier from aliases. - * 5. Try to resolve identifier from tables. + * 5. Try to resolve identifier from join tree. * * 6. If it is table identifier lookup try to lookup identifier in current scope CTEs. * @@ -1548,10 +1628,13 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook if (unlikely(prefer_column_name_to_alias)) { - resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + if (identifier_resolve_settings.allow_to_check_join_tree) + { + resolve_result.resolved_identifier = tryResolveIdentifierFromJoinTree(identifier_lookup, scope); - if (resolve_result.resolved_identifier) - resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + if (resolve_result.resolved_identifier) + resolve_result.resolve_place = IdentifierResolvePlace::JOIN_TREE; + } if (!resolve_result.resolved_identifier) { @@ -1569,12 +1652,12 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook { resolve_result.resolve_place = IdentifierResolvePlace::ALIASES; } - else + else if (identifier_resolve_settings.allow_to_check_join_tree) { - resolve_result.resolved_identifier = tryResolveIdentifierFromTables(identifier_lookup, scope); + resolve_result.resolved_identifier = tryResolveIdentifierFromJoinTree(identifier_lookup, scope); if (resolve_result.resolved_identifier) - resolve_result.resolve_place = IdentifierResolvePlace::TABLES; + resolve_result.resolve_place = IdentifierResolvePlace::JOIN_TREE; } } } @@ -1703,7 +1786,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (table_expression_query_node) { - initial_matcher_columns = getQueryNodeColumns(table_expression_node); + initial_matcher_columns = table_expression_query_node->computeProjectionColumns(); } else { @@ -1753,7 +1836,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, if (from_query_node) { - initial_matcher_columns = getQueryNodeColumns(scope_query_node->getFrom()); + initial_matcher_columns = from_query_node->computeProjectionColumns(); } else if (from_table_node || from_table_function_node) { @@ -1860,6 +1943,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * except_transformer = transformer->as()) { + auto node_name = node->getName(); if (except_transformer->isColumnMatching(node_name)) @@ -1873,6 +1957,7 @@ QueryTreeNodePtr QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, } else if (auto * replace_transformer = transformer->as()) { + auto node_name = node->getName(); auto replace_expression = replace_transformer->findReplacementExpression(node_name); if (!replace_expression) @@ -2004,8 +2089,8 @@ void QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_node, const Qu scope.scope_node->formatASTForErrorMessage()); /// Initialize aliases in lambda scope - ScopeAliasVisitorMatcher::Data data{scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); + QueryExpressionsAliasVisitor::Data data{scope}; + QueryExpressionsAliasVisitor visitor(data); visitor.visit(lambda.getExpression()); /** Replace lambda arguments with new arguments. @@ -2126,7 +2211,6 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc auto & function_argument = function_arguments[function_argument_index]; ColumnWithTypeAndName argument_column; - argument_column.name = function_argument->getName(); /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction * where function argument types are initialized with empty array of lambda arguments size. @@ -2562,8 +2646,8 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes case QueryTreeNodeType::COLUMN: { auto & column_function_node = node->as(); - if (column_function_node.hasAliasExpression()) - resolveExpressionNode(column_function_node.getAliasExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (column_function_node.hasExpression()) + resolveExpressionNode(column_function_node.getExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); break; } case QueryTreeNodeType::FUNCTION: @@ -2610,6 +2694,20 @@ void QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierRes break; } + case QueryTreeNodeType::ARRAY_JOIN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Array join is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + case QueryTreeNodeType::JOIN: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Join is not allowed {} in expression. In scope {}", + node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } } /** Update aliases after expression node was resolved. @@ -2673,133 +2771,400 @@ void QueryAnalyzer::resolveExpressionNodeList(QueryTreeNodePtr & node_list, Iden node_list = std::move(result_node_list); } -/** Initialize query FROM section. +/** Initialize query join tree node. + * + * 1. Resolve identifiers. + * 2. Register table, table function, query nodes in scope table expressions in resolve process. */ -void QueryAnalyzer::initializeQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope) { - if (from_node->getNodeType() == QueryTreeNodeType::QUERY || - from_node->getNodeType() == QueryTreeNodeType::TABLE) - { - /// Already initialized - } - else if (auto * table_function = from_node->as()) - { - /// Already initialized - } - else if (auto * from_table_identifier = from_node->as()) - { - auto table_identifier_lookup = IdentifierLookup{from_table_identifier->getIdentifier(), IdentifierLookupContext::TABLE}; - auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope); - auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; + std::deque join_tree_node_ptrs_to_process_queue; + join_tree_node_ptrs_to_process_queue.push_back(&join_tree_node); - if (!resolved_identifier) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {} identifier {} in scope {}", - toStringLowercase(IdentifierLookupContext::TABLE), - from_table_identifier->getIdentifier().getFullName(), - scope.scope_node->formatASTForErrorMessage()); + while (!join_tree_node_ptrs_to_process_queue.empty()) + { + auto * current_join_tree_node_ptr = join_tree_node_ptrs_to_process_queue.front(); + join_tree_node_ptrs_to_process_queue.pop_front(); - bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + auto & current_join_tree_node = *current_join_tree_node_ptr; + auto current_join_tree_node_type = current_join_tree_node->getNodeType(); - if (resolved_as_cte) + switch (current_join_tree_node_type) { - resolved_identifier = resolved_identifier->clone(); - resolved_identifier->as().setIsCTE(false); + case QueryTreeNodeType::IDENTIFIER: + { + auto & from_table_identifier = current_join_tree_node->as(); + auto table_identifier_lookup = IdentifierLookup{from_table_identifier.getIdentifier(), IdentifierLookupContext::TABLE}; + + IdentifierResolveSettings resolve_settings; + /// In join tree initialization ignore join tree as identifier lookup source + resolve_settings.allow_to_check_join_tree = false; + + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); + auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; + + if (!resolved_identifier) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown {} identifier {} in scope {}", + toStringLowercase(IdentifierLookupContext::TABLE), + from_table_identifier.getIdentifier().getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + /** SELECT a.id, b.id FROM test_table_join_1 AS a INNER JOIN test_table_join_1 AS b. + * Because of identifier cache we receive same result for same identifier. + * Use table_expressions_in_resolve_process to detect if resolved identifier must be cloned. + */ + if (scope.table_expressions_in_resolve_process.contains(resolved_identifier.get())) + { + resolved_identifier = resolved_identifier->clone(); + resolved_identifier->removeAlias(); + } + + bool resolved_as_cte = resolved_identifier->as() && resolved_identifier->as()->isCTE(); + + if (resolved_as_cte) + { + resolved_identifier = resolved_identifier->clone(); + resolved_identifier->as().setIsCTE(false); + } + + auto current_join_tree_node_alias = current_join_tree_node->getAlias(); + resolved_identifier->setAlias(current_join_tree_node_alias); + current_join_tree_node = resolved_identifier; + + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::QUERY: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::TABLE: + { + scope.table_expressions_in_resolve_process.insert(current_join_tree_node.get()); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join = current_join_tree_node->as(); + join_tree_node_ptrs_to_process_queue.push_back(&array_join.getTableExpression()); + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join = current_join_tree_node->as(); + join_tree_node_ptrs_to_process_queue.push_back(&join.getLeftTableExpression()); + join_tree_node_ptrs_to_process_queue.push_back(&join.getRightTableExpression()); + break; + } + default: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query FROM section expected table, table function, query, ARRAY JOIN or JOIN. Actual {} {}. In scope {}", + current_join_tree_node->getNodeTypeName(), + current_join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } } - - auto from_node_alias = from_node->getAlias(); - resolved_identifier->setAlias(from_node_alias); - from_node = resolved_identifier; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Query FROM section expected table or query. Actual {} {}. In scope {}", - from_node->getNodeTypeName(), - from_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - - auto from_node_alias = from_node->getAlias(); - if (!from_node_alias.empty()) - { - auto [_, inserted] = scope.alias_name_to_table_expression_node.insert(std::make_pair(from_node_alias, from_node)); - if (!inserted) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Duplicate aliases for tables in FROM section are not allowed", - from_node_alias); } } -/** Resolve query FROM section. - * - * Resolve steps: - * 1. If FROM node is query, resolve it. - * 2. If FROM node is table is it already resolved. - */ -void QueryAnalyzer::resolveQueryFrom(QueryTreeNodePtr & from_node, IdentifierResolveScope & scope) +/// Initialize table expression columns for table expression node +void QueryAnalyzer::initializeTableExpressionColumns(QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) { - if (from_node->getNodeType() == QueryTreeNodeType::QUERY) - { - IdentifierResolveScope subquery_scope(from_node, &scope); - resolveQuery(from_node, subquery_scope); - return; - } - else if (auto * from_table_function = from_node->as()) - { - const auto & table_function_factory = TableFunctionFactory::instance(); - const auto & table_function_name = from_table_function->getTableFunctionName(); + auto * table_node = table_expression_node->as(); + auto * query_node = table_expression_node->as(); + auto * table_function_node = table_expression_node->as(); - TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); - if (!table_function_ptr) + if (!table_node && !table_function_node && !query_node) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Unexpected table expression. Expected table, table function or query node. Actual {}. In scope {}", + table_expression_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + + auto storage_columns_it = table_expression_node_to_columns.find(table_expression_node.get()); + if (storage_columns_it != table_expression_node_to_columns.end()) + return; + + TableExpressionColumns storage_columns; + + if (table_node || table_function_node) + { + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + const auto & columns_description = storage_snapshot->metadata->getColumns(); + + std::vector> alias_columns_to_resolve; + std::unordered_map column_name_to_column_node; + column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) { - auto hints = TableFunctionFactory::instance().getHints(table_function_name); - if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function_name, toString(hints)); + const auto & column_default = columns_description.getDefault(column_name_and_type.name); + + if (column_default && column_default->kind == ColumnDefaultKind::Alias) + { + auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, context), table_expression_node); + column_name_to_column_node.emplace(column_name_and_type.name, column_node); + alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); + } else - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); - } - - if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) - { - const auto & insertion_table = context->getInsertionTable(); - if (!insertion_table.empty()) { - const auto & structure_hint = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; - table_function_ptr->setStructureHint(structure_hint); + auto column_node = std::make_shared(column_name_and_type, table_expression_node); + column_name_to_column_node.emplace(column_name_and_type.name, column_node); } } - /// TODO: Special functions that can take query - /// TODO: Support qualified matchers for table function - - for (auto & argument_node : from_table_function->getArguments().getNodes()) + for (auto & [alias_column_to_resolve_name, alias_column_to_resolve] : alias_columns_to_resolve) { - if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) + /** Alias column could be potentially resolved during resolve of other ALIAS column. + * Example: CREATE TABLE test_table (id UInt64, alias_value_1 ALIAS id + alias_value_2, alias_value_2 ALIAS id + 5) ENGINE=TinyLog; + * + * During resolve of alias_value_1, alias_value_2 column will be resolved. + */ + alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; + + IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); + alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); + + /// Initialize aliases in alias column scope + QueryExpressionsAliasVisitor::Data data{alias_column_resolve_scope}; + QueryExpressionsAliasVisitor visitor(data); + + visitor.visit(alias_column_to_resolve->getExpression()); + + resolveExpressionNode(alias_column_resolve_scope.scope_node, + alias_column_resolve_scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + + column_name_to_column_node = std::move(alias_column_resolve_scope.column_name_to_column_node); + column_name_to_column_node[alias_column_to_resolve_name] = alias_column_to_resolve; + } + + storage_columns.column_name_to_column_node = std::move(column_name_to_column_node); + } + else if (query_node) + { + auto column_names_and_types = query_node->computeProjectionColumns(); + storage_columns.column_name_to_column_node.reserve(column_names_and_types.size()); + + for (const auto & column_name_and_type : column_names_and_types) + { + auto column_node = std::make_shared(column_name_and_type, table_expression_node); + storage_columns.column_name_to_column_node.emplace(column_name_and_type.name, column_node); + } + } + + storage_columns.column_identifier_first_parts.reserve(storage_columns.column_name_to_column_node.size()); + + for (auto & [column_name, _] : storage_columns.column_name_to_column_node) + { + Identifier column_name_identifier(column_name); + storage_columns.column_identifier_first_parts.insert(column_name_identifier.at(0)); + } + + table_expression_node_to_columns.emplace(table_expression_node.get(), std::move(storage_columns)); +} + +/** Resolve query join tree. + * + * Query join tree must be initialized before calling this function. + */ +void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor) +{ + auto add_table_expression_alias_into_scope = [&](const QueryTreeNodePtr & table_expression_node) + { + const auto & alias_name = table_expression_node->getAlias(); + if (alias_name.empty()) + return; + + auto [it, inserted] = scope.alias_name_to_table_expression_node.emplace(alias_name, table_expression_node); + if (!inserted) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Duplicate aliases {} for table expressions in FROM section are not allowed. Try to register {}. Already registered {}.", + alias_name, + table_expression_node->formatASTForErrorMessage(), + it->second->formatASTForErrorMessage()); + }; + + auto from_node_type = join_tree_node->getNodeType(); + + switch (from_node_type) + { + case QueryTreeNodeType::QUERY: + { + IdentifierResolveScope subquery_scope(join_tree_node, &scope); + resolveQuery(join_tree_node, subquery_scope); + break; + } + case QueryTreeNodeType::TABLE_FUNCTION: + { + auto & table_function_node = join_tree_node->as(); + expressions_visitor.visit(table_function_node.getArgumentsNode()); + + const auto & table_function_factory = TableFunctionFactory::instance(); + const auto & table_function_name = table_function_node.getTableFunctionName(); + + TableFunctionPtr table_function_ptr = table_function_factory.tryGet(table_function_name, context); + if (!table_function_ptr) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Matcher as table function argument is not supported {}. In scope {}", - from_node->formatASTForErrorMessage(), + auto hints = TableFunctionFactory::instance().getHints(table_function_name); + if (!hints.empty()) + throw Exception( + ErrorCodes::UNKNOWN_FUNCTION, + "Unknown table function {}. Maybe you meant: {}", + table_function_name, + DB::toString(hints)); + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function_name); + } + + if (context->getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + { + const auto & insertion_table = context->getInsertionTable(); + if (!insertion_table.empty()) + { + const auto & structure_hint + = DatabaseCatalog::instance().getTable(insertion_table, context)->getInMemoryMetadataPtr()->columns; + table_function_ptr->setStructureHint(structure_hint); + } + } + + /// TODO: Special functions that can take query + /// TODO: Support qualified matchers for table function + + for (auto & argument_node : table_function_node.getArguments().getNodes()) + { + if (argument_node->getNodeType() == QueryTreeNodeType::MATCHER) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Matcher as table function argument is not supported {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + + resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + } + + auto table_function_ast = table_function_node.toAST(); + table_function_ptr->parseArguments(table_function_ast, context); + + auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); + table_function_node.resolve(std::move(table_function_ptr), std::move(table_function_storage), context); + break; + } + case QueryTreeNodeType::TABLE: + { + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + { + auto & array_join_node = join_tree_node->as(); + resolveQueryJoinTreeNode(array_join_node.getTableExpression(), scope, expressions_visitor); + + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto array_join_expression_alias = array_join_expression->getAlias(); + + if (!array_join_expression_alias.empty() && scope.alias_name_to_expression_node.contains(array_join_expression_alias)) + throw Exception( + ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "ARRAY JOIN expression {} with duplicate alias {}. In scope {}", + array_join_expression->formatASTForErrorMessage(), + array_join_expression_alias, + scope.scope_node->formatASTForErrorMessage()); + } + + /// Add array join expressions into scope + + expressions_visitor.visit(array_join_node.getJoinExpressionsNode()); + resolveExpressionNode(array_join_node.getJoinExpressionsNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + /// Wrap array join expressions into column nodes, where array join expression is inner expression. + + for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) + { + auto array_join_expression_alias = array_join_expression->getAlias(); + + auto result_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + + auto result_type = array_join_expression->getResultType(); + + if (!isArray(result_type)) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN requires expression with Array type. Actual {}. In scope {}", + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + array_join_expression->removeAlias(); + + result_type = assert_cast(*result_type).getNestedType(); + + auto array_join_column = std::make_shared( + NameAndTypePair{result_name, result_type}, array_join_expression, join_tree_node); + array_join_expression = std::move(array_join_column); + array_join_expression->setAlias(array_join_expression_alias); + + auto it = scope.alias_name_to_expression_node.find(array_join_expression_alias); + if (it != scope.alias_name_to_expression_node.end()) + it->second = array_join_expression; + } + + break; + } + case QueryTreeNodeType::JOIN: + { + auto & join_node = join_tree_node->as(); + + resolveQueryJoinTreeNode(join_node.getLeftTableExpression(), scope, expressions_visitor); + resolveQueryJoinTreeNode(join_node.getRightTableExpression(), scope, expressions_visitor); + + if (join_node.isUsingJoinExpression()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} with USING is unsupported. In scope {}", + join_node.formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + + if (join_node.getJoinExpression()) + { + expressions_visitor.visit(join_node.getJoinExpression()); + resolveExpressionNode(join_node.getJoinExpression(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } - resolveExpressionNode(argument_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + break; } + case QueryTreeNodeType::IDENTIFIER: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifiers in FROM section must be already resolved. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + default: + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query FROM section expected table, table function, query, ARRAY JOIN or JOIN. Actual {}. In scope {}", + join_tree_node->formatASTForErrorMessage(), + scope.scope_node->formatASTForErrorMessage()); + } + } - auto table_function_ast = from_table_function->toAST(); - table_function_ptr->parseArguments(table_function_ast, context); + if (isTableExpression(join_tree_node.get())) + initializeTableExpressionColumns(join_tree_node, scope); - auto table_function_storage = table_function_ptr->execute(table_function_ast, context, table_function_ptr->getName()); - from_table_function->resolve(std::move(table_function_ptr), std::move(table_function_storage), context); - } - else if (auto * from_table = from_node->as()) - { - return; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Query FROM section expected table or query. Actual {}", from_node->formatASTForErrorMessage()); - } + add_table_expression_alias_into_scope(join_tree_node); + scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } /** Resolve query. @@ -2823,13 +3188,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Initialize aliases in query node scope - ScopeAliasVisitorMatcher::Data data{scope}; - ScopeAliasVisitorMatcher::Visitor visitor(data); + QueryExpressionsAliasVisitor::Data data{scope}; + QueryExpressionsAliasVisitor visitor(data); - if (query_node_typed.getWithNode()) + if (!query_node_typed.getWith().getNodes().empty()) visitor.visit(query_node_typed.getWithNode()); - if (query_node_typed.getProjectionNode()) + if (!query_node_typed.getProjection().getNodes().empty()) visitor.visit(query_node_typed.getProjectionNode()); if (query_node_typed.getPrewhere()) @@ -2840,35 +3205,33 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Register CTE subqueries - if (query_node_typed.getWithNode()) + for (auto & node : query_node_typed.getWith().getNodes()) { - for (auto & node : query_node_typed.getWith().getNodes()) - { - auto * subquery_node = node->as(); - if (!subquery_node || !subquery_node->isCTE()) - continue; + auto * subquery_node = node->as(); + if (!subquery_node || !subquery_node->isCTE()) + continue; - const auto & cte_name = subquery_node->getCTEName(); - auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); - if (!inserted) - throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, - "CTE with name {} already exists. In scope {}", - cte_name, - data.scope.scope_node->formatASTForErrorMessage()); - } + const auto & cte_name = subquery_node->getCTEName(); + auto [_, inserted] = data.scope.cte_name_to_query_node.emplace(cte_name, node); + if (!inserted) + throw Exception( + ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS, + "CTE with name {} already exists. In scope {}", + cte_name, + data.scope.scope_node->formatASTForErrorMessage()); } if (query_node_typed.getFrom()) { - if (auto * table_function = query_node_typed.getFrom()->as()) - visitor.visit(table_function->getArgumentsNode()); + TableExpressionsAliasVisitor::Data table_expressions_visitor_data{scope}; + TableExpressionsAliasVisitor table_expressions_visitor(table_expressions_visitor_data); - auto [it, _] = query_node_from_section_in_resolve_process.emplace(query_node.get()); + table_expressions_visitor.visit(query_node_typed.getFrom()); - initializeQueryFrom(query_node_typed.getFrom(), scope); - resolveQueryFrom(query_node_typed.getFrom(), scope); + initializeQueryJoinTreeNode(query_node_typed.getFrom(), scope); + scope.alias_name_to_table_expression_node.clear(); - query_node_from_section_in_resolve_process.erase(it); + resolveQueryJoinTreeNode(query_node_typed.getFrom(), scope, visitor); } /** Clear cache of identifier lookups. @@ -2895,7 +3258,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.getWhere()) resolveExpressionNode(query_node_typed.getWhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - /** WITH section can be safely removed, because WITH section only can provide aliases to expressions + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); @@ -2913,7 +3276,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { auto node = node_with_duplicated_alias; auto node_alias = node->getAlias(); - resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); + resolveExpressionNode(node, scope, true /*allow_lambda_expression*/, true /*allow_table_expression*/); bool has_node_in_alias_table = false; @@ -2954,6 +3317,8 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier } } +} + void QueryAnalysisPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { if (query_tree_node->getNodeType() != QueryTreeNodeType::QUERY) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 83a2d6d2698..36972fed934 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -1,16 +1,22 @@ #include +#include + +#include + +#include +#include +#include + #include #include #include #include #include +#include +#include -#include - -#include -#include -#include +#include namespace DB { @@ -22,6 +28,29 @@ QueryNode::QueryNode() children[projection_child_index] = std::make_shared(); } +NamesAndTypesList QueryNode::computeProjectionColumns() const +{ + NamesAndTypes query_columns; + + const auto & projection_nodes = getProjection(); + query_columns.reserve(projection_nodes.getNodes().size()); + + for (const auto & projection_node : projection_nodes.getNodes()) + { + auto column_type = projection_node->getResultType(); + std::string column_name; + + if (projection_node->hasAlias()) + column_name = projection_node->getAlias(); + else + column_name = projection_node->getName(); + + query_columns.emplace_back(column_name, column_type); + } + + return {query_columns.begin(), query_columns.end()}; +} + String QueryNode::getName() const { WriteBufferFromOwnString buffer; @@ -121,21 +150,8 @@ ASTPtr QueryNode::toASTImpl() const select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST()); - auto table_expression_ast = std::make_shared(); - table_expression_ast->children.push_back(children[from_child_index]->toAST()); - - if (children[from_child_index]->getNodeType() == QueryTreeNodeType::TABLE) - table_expression_ast->database_and_table_name = table_expression_ast->children.back(); - else if (children[from_child_index]->getNodeType() == QueryTreeNodeType::QUERY) - table_expression_ast->subquery = table_expression_ast->children.back(); - - auto tables_in_select_query_element_ast = std::make_shared(); - tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); - tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back(); - - auto tables_in_select_query_ast = std::make_shared(); - tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast)); - + ASTPtr tables_in_select_query_ast = std::make_shared(); + addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[from_child_index]); select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast)); if (getPrewhere()) diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 78941cb2cae..a3b9ebaedd6 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -107,6 +107,11 @@ public: return children[prewhere_child_index]; } + bool hasWhere() const + { + return children[where_child_index] != nullptr; + } + const QueryTreeNodePtr & getWhere() const { return children[where_child_index]; @@ -117,6 +122,9 @@ public: return children[where_child_index]; } + /// Compute query node columns using projection section + NamesAndTypesList computeProjectionColumns() const; + QueryTreeNodeType getNodeType() const override { return QueryTreeNodeType::QUERY; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 2385c8e74d2..cf6f541cbfb 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -31,6 +31,8 @@ #include #include #include +#include +#include #include #include @@ -360,8 +362,8 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q } auto & tables = tables_in_select_query->as(); - if (tables.children.size() > 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple tables are not supported"); + + QueryTreeNodes table_expressions; for (const auto & table_element_untyped : tables.children) { @@ -380,7 +382,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(table_identifier_typed.tryGetAlias()); node->setOriginalAST(table_element.table_expression); - return node; + table_expressions.push_back(std::move(node)); } else if (table_expression.subquery) { @@ -392,7 +394,7 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(subquery_expression.tryGetAlias()); node->setOriginalAST(select_with_union_query); - return node; + table_expressions.push_back(std::move(node)); } else if (table_expression.table_function) { @@ -410,59 +412,68 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q node->setAlias(table_function_expression.tryGetAlias()); node->setOriginalAST(table_expression.table_function); - return node; + table_expressions.push_back(std::move(node)); + } + else + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}", table_element.table_expression->formatForErrorMessage()); } - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}", table_element.table_expression->formatForErrorMessage()); } - // if (table_element.table_join) - // { - // const auto & table_join = table_element.table_join->as(); + if (table_element.table_join) + { + const auto & table_join = table_element.table_join->as(); - // auto right_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto right_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto left_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto left_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto join_expression = JoinExpression::create(); - // join_expression->getLeftTableExpression() = left_table_expression; - // join_expression->getRightTableExpression() = right_table_expression; + QueryTreeNodePtr join_expression; - // if (table_join->using_expression_list) - // join_expression->getUsingExpressions() = getExpressionElements(table_join->using_expression_list, scope); + if (table_join.using_expression_list) + join_expression = getExpressionList(table_join.using_expression_list); + else if (table_join.on_expression) + join_expression = getExpression(table_join.on_expression); - // if (table_join->on_expression) - // { - // join_expression->getOnExpression() = getExpressionElement(table_join->on_expression, scope); - // } + auto join_node = std::make_shared(std::move(left_table_expression), + std::move(right_table_expression), + std::move(join_expression), + table_join.locality, + table_join.strictness, + table_join.kind); - // table_expressions.emplace_back(std::move(join_expression)); - // } + /** Original AST is not set because it will contain only join part and does + * not include left table expression. + */ + table_expressions.emplace_back(std::move(join_node)); + } - // if (table_element.array_join) - // { - // auto array_join_array_expression = table_element.array_join->children[0]->children[0]; - // auto expression_element = getExpressionElement(array_join_array_expression, scope); - // expression_element->setAlias(array_join_array_expression->tryGetAlias()); + if (table_element.array_join) + { + auto & array_join_expression = table_element.array_join->as(); + bool is_left_array_join = array_join_expression.kind == ASTArrayJoin::Kind::Left; - // auto last_table_expression = std::move(table_expressions.back()); - // table_expressions.pop_back(); + auto last_table_expression = std::move(table_expressions.back()); + table_expressions.pop_back(); - // auto array_join_expression = ArrayJoinExpression::create(); - // array_join_expression->getLeftTableExpression() = std::move(last_table_expression); - // array_join_expression->getArrayExpression() = std::move(expression_element); + auto array_join_expressions_list = getExpressionList(array_join_expression.expression_list); - // table_expressions.push_back(array_join_expression); - // } + auto array_join_node = std::make_shared(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join); + array_join_node->setOriginalAST(table_element.array_join); + + table_expressions.push_back(std::move(array_join_node)); + } } - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From section is unsupported"); - // if (table_expressions.empty()) - // throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryAnalyzer from cannot be empty"); + if (table_expressions.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot be empty"); - // return table_expressions.back(); + if (table_expressions.size() > 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot have more than 1 root table expression"); + + return table_expressions.back(); } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index cc5de07b916..768e6f0e6a3 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -13,11 +13,11 @@ namespace DB { -TableNode::TableNode(StoragePtr storage_, ContextPtr context) +TableNode::TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_) : storage(std::move(storage_)) , storage_id(storage->getStorageID()) - , table_lock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)) - , storage_snapshot(storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context)) + , storage_lock(std::move(storage_lock_)) + , storage_snapshot(std::move(storage_snapshot_)) { } @@ -62,7 +62,7 @@ QueryTreeNodePtr TableNode::cloneImpl() const result_table_node->storage = storage; result_table_node->storage_id = storage_id; - result_table_node->table_lock = table_lock; + result_table_node->storage_lock = storage_lock; result_table_node->storage_snapshot = storage_snapshot; return result_table_node; diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 6756381ed34..0a528b4db67 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -26,8 +26,8 @@ using TableNodePtr = std::shared_ptr; class TableNode : public IQueryTreeNode { public: - /// Construct table node with storage and context - explicit TableNode(StoragePtr storage_, ContextPtr context); + /// Construct table node with storage, storage lock, storage snapshot + explicit TableNode(StoragePtr storage_, TableLockHolder storage_lock_, StorageSnapshotPtr storage_snapshot_); /// Get storage const StoragePtr & getStorage() const @@ -47,18 +47,18 @@ public: return storage_snapshot; } - /// Get table lock - const TableLockHolder & getTableLock() const + /// Get storage lock + const TableLockHolder & getStorageLock() const { - return table_lock; + return storage_lock; } /** Move table lock out of table node. * After using this method table node state becomes invalid. */ - TableLockHolder && moveTableLock() + TableLockHolder && moveStorageLock() { - return std::move(table_lock); + return std::move(storage_lock); } QueryTreeNodeType getNodeType() const override @@ -84,7 +84,7 @@ private: StoragePtr storage; StorageID storage_id; - TableLockHolder table_lock; + TableLockHolder storage_lock; StorageSnapshotPtr storage_snapshot; }; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 1aade42351f..75b3f37aaad 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -1,5 +1,12 @@ #include +#include +#include +#include +#include + +#include + namespace DB { @@ -36,4 +43,91 @@ bool isNameOfInFunction(const std::string & function_name) return is_special_function_in; } +bool isTableExpression(const IQueryTreeNode * node) +{ + auto node_type = node->getNodeType(); + return node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::TABLE_FUNCTION || node_type == QueryTreeNodeType::QUERY; +} + +static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node) +{ + ASTPtr table_expression_node_ast; + auto node_type = table_expression_node->getNodeType(); + + if (node_type == QueryTreeNodeType::IDENTIFIER) + { + const auto & identifier_node = table_expression_node->as(); + const auto & identifier = identifier_node.getIdentifier(); + + if (identifier.getPartsSize() == 1) + table_expression_node_ast = std::make_shared(identifier[0]); + else if (identifier.getPartsSize() == 2) + table_expression_node_ast = std::make_shared(identifier[0], identifier[1]); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Identifier for table expression must contain 1 or 2 parts. Actual {}", + identifier.getFullName()); + } + else + { + table_expression_node_ast = table_expression_node->toAST(); + } + + auto result_table_expression = std::make_shared(); + result_table_expression->children.push_back(table_expression_node_ast); + + if (node_type == QueryTreeNodeType::QUERY) + result_table_expression->subquery = result_table_expression->children.back(); + else if (node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::IDENTIFIER) + result_table_expression->database_and_table_name = result_table_expression->children.back(); + else if (node_type == QueryTreeNodeType::TABLE_FUNCTION) + result_table_expression->table_function = result_table_expression->children.back(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected identiifer, table, query, or table function. Actual {}", table_expression_node->formatASTForErrorMessage()); + + return result_table_expression; +} + +void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression) +{ + auto table_expression_node_type = table_expression->getNodeType(); + + switch (table_expression_node_type) + { + case QueryTreeNodeType::IDENTIFIER: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + auto table_expression_ast = convertIntoTableExpressionAST(table_expression); + + auto tables_in_select_query_element_ast = std::make_shared(); + tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast)); + tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back(); + + tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast)); + break; + } + case QueryTreeNodeType::ARRAY_JOIN: + [[fallthrough]]; + case QueryTreeNodeType::JOIN: + { + auto table_expression_tables_in_select_query_ast = table_expression->toAST(); + tables_in_select_query_ast->children.reserve(table_expression_tables_in_select_query_ast->children.size()); + for (auto && left_table_element_ast : table_expression_tables_in_select_query_ast->children) + tables_in_select_query_ast->children.push_back(std::move(left_table_element_ast)); + break; + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected node type for table expression. Expected table, query, table function, join or array join. Actual {}", + table_expression->getNodeTypeName()); + } + } +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index d93d81d654e..015233a05dc 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -11,4 +11,12 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); /// Returns true if function name is name of IN function or its variations, false otherwise bool isNameOfInFunction(const std::string & function_name); +/// Returns true if node hase type table, table function, or query, false otherwise +bool isTableExpression(const IQueryTreeNode * node); + +/** Add table expression in tables in select query children. + * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. + */ +void addTableExpressionIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression); + } diff --git a/src/Analyzer/examples/query_analyzer.cpp b/src/Analyzer/examples/query_analyzer.cpp index 0e9073306a0..5a20b46b346 100644 --- a/src/Analyzer/examples/query_analyzer.cpp +++ b/src/Analyzer/examples/query_analyzer.cpp @@ -1,7 +1,5 @@ #include -using namespace DB; - int main(int argc, char ** argv) { (void)(argc); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index b27df0f1c35..9b38072b5af 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1073,8 +1073,8 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ } /// Result will also contain joined columns. - for (const auto & column_name : analyzed_join->columnsAddedByJoin()) - required_names.emplace(column_name); + for (const auto & column : analyzed_join->columnsAddedByJoin()) + required_names.emplace(column.name); for (const auto & column : result_columns) { diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index baba2214333..d3d13d80f57 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -417,8 +418,17 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); - interpreter.buildQueryPlan(plan); + if (getContext()->getSettingsRef().use_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + interpreter.initializeQueryPlanIfNeeded(); + plan = std::move(interpreter).extractQueryPlan(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); + interpreter.buildQueryPlan(plan); + } if (settings.optimize) plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); @@ -452,8 +462,18 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); - interpreter.buildQueryPlan(plan); + if (getContext()->getSettingsRef().use_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + interpreter.initializeQueryPlanIfNeeded(); + plan = std::move(interpreter).extractQueryPlan(); + } + else + { + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); + interpreter.buildQueryPlan(plan); + } + auto pipeline = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(getContext()), BuildQueryPipelineSettings::fromContext(getContext())); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index b407a77c50c..c2b2d95a94d 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -21,11 +22,14 @@ #include #include #include +#include +#include #include #include #include #include +#include #include #include @@ -34,9 +38,14 @@ #include #include #include +#include +#include #include #include +#include +#include +#include namespace DB @@ -48,33 +57,37 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int INVALID_JOIN_ON_EXPRESSION; } -QueryPipeline buildDummyPipeline() +/** ClickHouse query planner. + * + * TODO: JOIN support columns cast. JOIN support ASOF. JOIN support strictness. + * TODO: Support RBAC. Support RBAC for ALIAS columns. + * TODO: Support distributed query processing + * TODO: Support PREWHERE + * TODO: Support GROUP BY, HAVING + * TODO: Support ORDER BY, LIMIT + * TODO: Support WINDOW FUNCTIONS + * TODO: Support DISTINCT + * TODO: Support ArrayJoin + * TODO: Support building sets for IN functions + * TODO: Support trivial count optimization + * TODO: Support totals, extremes + * TODO: Support projections + */ + +namespace { - ColumnsWithTypeAndName columns; - auto string_data_type = std::make_shared(); - auto string_column = string_data_type->createColumn(); - string_column->insert("TestValue"); - columns.emplace_back(ColumnWithTypeAndName{std::move(string_column), string_data_type, "test_column"}); - - Block block(columns); - auto source = std::make_shared(block); - auto shell_input_pipe = Pipe(std::move(source)); - - QueryPipeline pipeline(std::move(shell_input_pipe)); - return pipeline; -} - -String dumpQueryPlan(QueryPlan & query_plan) +[[maybe_unused]] String dumpQueryPlan(QueryPlan & query_plan) { WriteBufferFromOwnString query_plan_buffer; - query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{}); + query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{true, true, true, true}); return query_plan_buffer.str(); } -String dumpQueryPipeline(QueryPlan & query_plan) +[[maybe_unused]] String dumpQueryPipeline(QueryPlan & query_plan) { QueryPlan::ExplainPipelineOptions explain_pipeline; WriteBufferFromOwnString query_pipeline_buffer; @@ -82,6 +95,434 @@ String dumpQueryPipeline(QueryPlan & query_plan) return query_pipeline_buffer.str(); } +struct TableExpressionColumns +{ + NamesAndTypesList all_columns; + NameSet all_columns_names_set; + + NamesAndTypesList source_input_columns; + NameSet source_columns_set; + std::unordered_map column_name_to_column_identifier; +}; + +using TableExpressionNodeToColumns = std::unordered_map; +using TableExpressionColumnNodeToColumnIdentifier = std::unordered_map; +using ActionsNodeNameToCount = std::unordered_map; + +class ActionsChainNode; +using ActionsChainNodePtr = std::unique_ptr; +using ActionsChainNodes = std::vector; + +class ActionsChainNode +{ +public: + explicit ActionsChainNode(ActionsDAGPtr actions_, bool available_output_columns_only_aliases_ = false) + : actions(std::move(actions_)) + , available_output_columns_only_aliases(available_output_columns_only_aliases_) + { + initialize(); + } + + [[maybe_unused]] ActionsDAGPtr & getActions() + { + return actions; + } + + [[maybe_unused]] const ActionsDAGPtr & getActions() const + { + return actions; + } + + const ColumnsWithTypeAndName & getAvailableOutputColumns() const + { + return available_output_columns; + } + + const NameSet & getInputColumnNames() const + { + return input_columns_names; + } + + const NameSet & getChildRequiredOutputColumnsNames() const + { + return child_required_output_columns_names; + } + + void finalizeInputAndOutputColumns(NameSet & child_input_columns) + { + child_required_output_columns_names.clear(); + std::vector required_output_nodes; + + for (const auto & node : actions->getNodes()) + { + auto it = child_input_columns.find(node.result_name); + + if (it == child_input_columns.end()) + continue; + + child_required_output_columns_names.insert(node.result_name); + required_output_nodes.push_back(&node); + child_input_columns.erase(it); + } + + for (auto & required_output_node : required_output_nodes) + actions->addOrReplaceInOutputs(*required_output_node); + + actions->removeUnusedActions(); + initialize(); + } + + void dump(WriteBuffer & buffer) const + { + buffer << "DAG" << '\n'; + buffer << actions->dumpDAG(); + if (!child_required_output_columns_names.empty()) + { + buffer << "Child required output columns " << boost::join(child_required_output_columns_names, ", "); + buffer << '\n'; + } + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } + + // NamesAndTypes getAvailableOutputNamesAndTypes() const + // { + // NamesAndTypes result; + // result.reserve(available_output_columns.size()); + + // for (const auto & available_output_column : available_output_columns) + // result.emplace_back(available_output_column.name, available_output_column.type); + + // return result; + // } + + // [[maybe_unused]] Names getAvailableOutputNames() const + // { + // Names result; + // result.reserve(available_output_columns.size()); + + // for (const auto & available_output_column : available_output_columns) + // result.emplace_back(available_output_column.name); + + // return result; + // } + + [[maybe_unused]] void addParentIndex(size_t parent_node_index) + { + parent_nodes_indices.push_back(parent_node_index); + } + + void addParentIndices(const std::vector & parent_nodes_indices_value) + { + parent_nodes_indices.insert(parent_nodes_indices.end(), parent_nodes_indices_value.begin(), parent_nodes_indices_value.end()); + } + + const std::vector & getParentNodesIndices() const + { + return parent_nodes_indices; + } + +private: + void initialize() + { + auto required_columns_names = actions->getRequiredColumnsNames(); + input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); + + available_output_columns.clear(); + + for (const auto & node : actions->getNodes()) + { + if (available_output_columns_only_aliases) + { + if (node.type == ActionsDAG::ActionType::ALIAS) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + + continue; + } + + if (node.type == ActionsDAG::ActionType::INPUT || + node.type == ActionsDAG::ActionType::FUNCTION || + node.type == ActionsDAG::ActionType::ARRAY_JOIN) + available_output_columns.emplace_back(node.column, node.result_type, node.result_name); + } + } + + ActionsDAGPtr actions; + + bool available_output_columns_only_aliases; + + NameSet input_columns_names; + + NameSet child_required_output_columns_names; + + ColumnsWithTypeAndName available_output_columns; + + std::vector parent_nodes_indices; + +}; + +class ActionsChain +{ +public: + void addNode(ActionsChainNodePtr node) + { + nodes.emplace_back(std::move(node)); + } + + [[maybe_unused]] const ActionsChainNodes & getNodes() const + { + return nodes; + } + + ColumnsWithTypeAndName getAvailableOutputColumns(const std::vector & nodes_indices) + { + ColumnsWithTypeAndName result; + + for (const auto & node_index : nodes_indices) + { + assert(node_index < nodes.size()); + const auto & node_available_output_columns = nodes[node_index]->getAvailableOutputColumns(); + result.insert(result.end(), node_available_output_columns.begin(), node_available_output_columns.end()); + } + + return result; + } + + // ColumnsWithTypeAndName getOutputColumns(const std::vector & nodes_indices) + // { + // ColumnsWithTypeAndName result; + + // for (const auto & node_index : nodes_indices) + // { + // assert(node_index < nodes.size()); + // const auto & node_output_columns = nodes[node_index]->getActions()->getResultColumns(); + // result.insert(result.end(), node_output_columns.begin(), node_output_columns.end()); + // } + + // return result; + // } + + [[maybe_unused]] NameSet getInputColumnNames(const std::vector & nodes_indices) + { + NameSet result; + + for (const auto & node_index : nodes_indices) + { + assert(node_index < nodes.size()); + const auto & node_input_column_names = nodes[node_index]->getInputColumnNames(); + result.insert(node_input_column_names.begin(), node_input_column_names.end()); + } + + return result; + } + + [[maybe_unused]] size_t size() const + { + return nodes.size(); + } + + [[maybe_unused]] const ActionsChainNodePtr & at(size_t index) const + { + if (index >= nodes.size()) + throw std::out_of_range("actions chain access is out of range"); + + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNodePtr & at(size_t index) + { + if (index >= nodes.size()) + throw std::out_of_range("actions chain access is out of range"); + + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNodePtr & operator[](size_t index) + { + return nodes[index]; + } + + [[maybe_unused]] const ActionsChainNodePtr & operator[](size_t index) const + { + return nodes[index]; + } + + [[maybe_unused]] ActionsChainNode * getLastNode() + { + return nodes.back().get(); + } + + [[maybe_unused]] ActionsChainNode * getLastNodeOrThrow() + { + if (nodes.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return nodes.back().get(); + } + + size_t getLastNodeIndex() + { + return nodes.size() - 1; + } + + [[maybe_unused]] size_t getLastNodeIndexOrThrow() + { + if (nodes.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ActionsChain is empty"); + + return nodes.size() - 1; + } + + void finalize() + { + if (nodes.empty()) + return; + + std::deque nodes_indices_to_process; + nodes_indices_to_process.push_front(nodes.size() - 1); + + /// For root node there are no columns required in child nodes + NameSet empty_child_input_columns; + nodes.back().get()->finalizeInputAndOutputColumns(empty_child_input_columns); + + while (!nodes_indices_to_process.empty()) + { + auto node_index_to_process = nodes_indices_to_process.front(); + nodes_indices_to_process.pop_front(); + + auto & node_to_process = nodes[node_index_to_process]; + + const auto & parent_nodes_indices = node_to_process->getParentNodesIndices(); + auto input_columns_names_copy = node_to_process->getInputColumnNames(); + + for (const auto & parent_node_index : parent_nodes_indices) + { + assert(parent_node_index < nodes.size()); + + auto & parent_node = nodes[parent_node_index]; + parent_node->finalizeInputAndOutputColumns(input_columns_names_copy); + nodes_indices_to_process.push_back(parent_node_index); + } + } + } + + void dump(WriteBuffer & buffer) const + { + size_t nodes_size = nodes.size(); + + for (size_t i = 0; i < nodes_size; ++i) + { + const auto & node = nodes[i]; + buffer << "Node " << i; + + const auto & parent_nodes_indices = node->getParentNodesIndices(); + if (!parent_nodes_indices.empty()) + { + buffer << " parent nodes indices "; + for (const auto & parent_node_index : parent_nodes_indices) + buffer << parent_node_index << ' '; + } + + buffer << '\n'; + node->dump(buffer); + + buffer << '\n'; + } + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + return buffer.str(); + } + +private: + ActionsChainNodes nodes; +}; + +class QueryPlanBuilder +{ +public: + using BuildRootStep = std::function; + using UniteStep = std::function)>; + using BuildStep = std::function; + + explicit QueryPlanBuilder(QueryPlan plan_root_) + { + auto plan_root_ptr = std::make_shared(std::move(plan_root_)); + build_root_step = [plan_root_ptr]() + { + return std::move(*plan_root_ptr); + }; + } + + [[maybe_unused]] explicit QueryPlanBuilder(std::vector plan_builders_, UniteStep unit_step_) + { + auto plan_builders_ptr = std::make_shared>(std::move(plan_builders_)); + build_root_step = [plan_builders_ptr, unite_step = std::move(unit_step_)]() + { + auto plan_builders = std::move(*plan_builders_ptr); + std::vector plans; + plans.reserve(plan_builders.size()); + + for (auto && plan_builder : plan_builders) + plans.push_back(std::move(plan_builder).buildPlan()); + + return unite_step(std::move(plans)); + }; + } + + QueryPlanBuilder(QueryPlanBuilder &&) noexcept = default; + [[maybe_unused]] QueryPlanBuilder & operator=(QueryPlanBuilder &&) noexcept = default; + + void addBuildStep(BuildStep step) + { + build_steps.push_back(std::move(step)); + } + + QueryPlan buildPlan() && + { + auto plan = build_root_step(); + + for (auto & build_step : build_steps) + build_step(plan); + + return plan; + } +private: + BuildRootStep build_root_step; + std::vector build_steps; +}; + +struct PlannerContext +{ + TableExpressionColumnNodeToColumnIdentifier table_expression_column_node_to_column_identifier; + TableExpressionNodeToColumns table_expression_node_to_columns; + size_t column_identifier_counter = 0; + + ActionsChain actions_chain; + + ActionsDAGPtr where_actions; + std::string where_action_node_name; + ActionsDAGPtr projection_actions; + + ContextPtr query_context; + + std::string getColumnUniqueIdentifier() + { + auto result = "__column_" + std::to_string(column_identifier_counter); + ++column_identifier_counter; + + return result; + } +}; struct QueryTreeActionsScopeNode { @@ -93,19 +534,19 @@ struct QueryTreeActionsScopeNode node_name_to_node[node.result_name] = &node; } - bool containsNode(const std::string & node_name) - { - return node_name_to_node.find(node_name) != node_name_to_node.end(); - } + // bool containsNode(const std::string & node_name) + // { + // return node_name_to_node.find(node_name) != node_name_to_node.end(); + // } - const ActionsDAG::Node * tryGetNode(const std::string & node_name) - { - auto it = node_name_to_node.find(node_name); - if (it == node_name_to_node.end()) - return {}; + // const ActionsDAG::Node * tryGetNode(const std::string & node_name) + // { + // auto it = node_name_to_node.find(node_name); + // if (it == node_name_to_node.end()) + // return {}; - return it->second; - } + // return it->second; + // } const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name) { @@ -196,13 +637,11 @@ struct QueryTreeActionsScopeNode QueryTreeNodePtr scope_node; }; -class QueryTreeActionsVisitor : public WithContext +class QueryTreeActionsVisitor { public: - explicit QueryTreeActionsVisitor( - ActionsDAGPtr actions_dag, - ContextPtr context_) - : WithContext(context_) + explicit QueryTreeActionsVisitor(ActionsDAGPtr actions_dag, const PlannerContext & planner_context_) + : planner_context(planner_context_) { actions_stack.emplace_back(std::move(actions_dag), nullptr); } @@ -230,111 +669,92 @@ public: private: - std::pair visitImpl(QueryTreeNodePtr node) + using NodeNameAndNodeMinLevel = std::pair; + + NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node) { if (auto * column_node = node->as()) - return visitColumn(*column_node); + return visitColumn(node); else if (auto * constant_node = node->as()) - return visitConstant(*constant_node); + return visitConstant(node); else if (auto * function_node = node->as()) - return visitFunction(*function_node); + return visitFunction(node); throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", node->formatASTForErrorMessage()); } - std::pair visitColumn(ColumnNode & column) + NodeNameAndNodeMinLevel visitColumn(const QueryTreeNodePtr & node) { - const auto & column_name = column.getColumnName(); + auto column_node_name = getActionsDAGNodeName(node.get()); + const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); for (Int64 i = actions_stack_size; i >= 0; --i) { - actions_stack[i].addInputColumnIfNecessary(column_name, column.getColumnType()); + actions_stack[i].addInputColumnIfNecessary(column_node_name, column_node.getColumnType()); - if (column.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && - actions_stack[i].scope_node.get() == column.getColumnSource().get()) + if (column_node.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA && + actions_stack[i].scope_node.get() == column_node.getColumnSource().get()) { - return {column_name, i}; + return {column_node_name, i}; } } - return {column_name, 0}; + return {column_node_name, 0}; } - std::pair visitConstant(ConstantNode & constant_node) + NodeNameAndNodeMinLevel visitConstant(const QueryTreeNodePtr & node) { + auto constant_node_name = getActionsDAGNodeName(node.get()); + const auto & constant_node = node->as(); const auto & literal = constant_node.getConstantValue(); - auto constant_name = constant_node.getName(); - ColumnWithTypeAndName column; - column.name = constant_name; + column.name = constant_node_name; column.type = constant_node.getResultType(); column.column = column.type->createColumnConst(1, literal); - actions_stack[0].addConstantIfNecessary(constant_name, column); + actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(constant_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {constant_name, 0}; + return {constant_node_name, 0}; } - std::pair visitLambda(QueryTreeNodePtr lambda_node_untyped) + NodeNameAndNodeMinLevel visitLambda(const QueryTreeNodePtr & node) { - auto & lambda_node = lambda_node_untyped->as(); + auto & lambda_node = node->as(); auto result_type = lambda_node.getResultType(); if (!result_type) throw Exception(ErrorCodes::LOGICAL_ERROR, "Lambda {} is not resolved during query analysis", lambda_node.formatASTForErrorMessage()); - // std::cout << "QueryTreeActionsVisitor::visitLambda " << lambda_node.formatASTForErrorMessage() << std::endl; - // std::cout << "Lambda arguments nodes size " << lambda_node.getArguments().getNodes().size() << std::endl; - NamesAndTypesList lambda_arguments_names_and_types; for (auto & lambda_node_argument : lambda_node.getArguments().getNodes()) { auto lambda_argument_name = lambda_node_argument->getName(); auto lambda_argument_type = lambda_node_argument->getResultType(); - // std::cout << "Lambda argument name " << lambda_argument_name; - // std::cout << " type " << lambda_argument_type->getName() << std::endl; lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type); } size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1; auto lambda_actions_dag = std::make_shared(); - actions_stack.emplace_back(lambda_actions_dag, lambda_node_untyped); + actions_stack.emplace_back(lambda_actions_dag, node); - auto [node_name, level] = visitImpl(lambda_node.getExpression()); - auto lambda_result_node_name = node_name; - lambda_actions_dag->getIndex().push_back(actions_stack.back().getNodeOrThrow(node_name)); - - // std::cout << "Previous DAG nodes " << actions_stack[previous_scope_node_actions_stack_index].actions_dag.get() << std::endl; - // for (const auto & previous_actions_node : actions_stack[previous_scope_node_actions_stack_index].actions_dag->getNodes()) - // { - // std::cout << "Node " << &previous_actions_node << " result name " << previous_actions_node.result_name << std::endl; - // std::cout << "Children " << previous_actions_node.children.size() << std::endl; - // for (const auto * child_node : previous_actions_node.children) - // { - // std::cout << "Child node " << child_node << " result name " << child_node->result_name << std::endl; - // } - // } - - lambda_actions_dag->removeUnusedActions(Names(1, lambda_result_node_name)); - - // std::cout << "Lambda actions DAG Node " << node_name << " level " << level << std::endl; - // std::cout << "Lambda actions DAG " << lambda_actions_dag.get() << std::endl; - // std::cout << lambda_actions_dag->dumpDAG() << std::endl; + auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression()); + lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); + lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); auto lambda_actions = std::make_shared( - lambda_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + lambda_actions_dag, ExpressionActionsSettings::fromContext(planner_context.query_context, CompileExpressions::yes)); Names captured_column_names; ActionsDAG::NodeRawConstPtrs lambda_children; @@ -354,31 +774,30 @@ private: } } - auto lambda_hash = lambda_node.getTreeHash(); - std::string lambda_name = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); - + auto lambda_node_name = getActionsDAGNodeName(node.get()); auto function_capture = std::make_shared( - lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_result_node_name); + lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_expression_node_name); actions_stack.pop_back(); if (level == actions_stack.size()) --level; - actions_stack[level].addFunctionIfNecessary(lambda_name, lambda_children, function_capture); + actions_stack[level].addFunctionIfNecessary(lambda_node_name, lambda_children, function_capture); size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputColumnIfNecessary(lambda_name, result_type); + actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type); } - return {lambda_name, level}; + return {lambda_node_name, level}; } - std::pair visitFunction(FunctionNode & function_node) + NodeNameAndNodeMinLevel visitFunction(const QueryTreeNodePtr & node) { - auto function_node_name = function_node.getName(); + auto function_node_name = getActionsDAGNodeName(node.get()); + const auto & function_node = node->as(); if (function_node.getFunctionName() == "grouping") { @@ -409,14 +828,14 @@ private: return {function_node_name, 0}; } - auto & function_arguments = function_node.getArguments().getNodes(); + const auto & function_arguments = function_node.getArguments().getNodes(); size_t function_arguments_size = function_arguments.size(); Names function_arguments_node_names; function_arguments_node_names.reserve(function_arguments_size); size_t level = 0; - for (auto & argument : function_arguments) + for (const auto & argument : function_arguments) { if (argument->getNodeType() == QueryTreeNodeType::LAMBDA) { @@ -461,7 +880,98 @@ private: return {function_node_name, level}; } + String getActionsDAGNodeName(const IQueryTreeNode * node) const + { + String result; + auto node_type = node->getNodeType(); + + switch (node_type) + { + case QueryTreeNodeType::COLUMN: + { + auto it = planner_context.table_expression_column_node_to_column_identifier.find(node); + if (it == planner_context.table_expression_column_node_to_column_identifier.end()) + return node->getName(); + + result = it->second; + break; + } + case QueryTreeNodeType::CONSTANT: + { + result = "__constant_" + node->getName(); + break; + } + case QueryTreeNodeType::FUNCTION: + { + const auto & function_node = node->as(); + + WriteBufferFromOwnString buffer; + buffer << function_node.getFunctionName(); + + const auto & function_parameters_nodes = function_node.getParameters().getNodes(); + + if (!function_parameters_nodes.empty()) + { + buffer << '('; + + size_t function_parameters_nodes_size = function_parameters_nodes.size(); + for (size_t i = 0; i < function_parameters_nodes_size; ++i) + { + const auto & function_parameter_node = function_parameters_nodes[i]; + getActionsDAGNodeName(function_parameter_node.get()); + + if (i + 1 != function_parameters_nodes_size) + buffer << ", "; + } + + buffer << ')'; + } + + const auto & function_arguments_nodes = function_node.getArguments().getNodes(); + + buffer << '('; + + size_t function_arguments_nodes_size = function_arguments_nodes.size(); + for (size_t i = 0; i < function_arguments_nodes_size; ++i) + { + const auto & function_argument_node = function_arguments_nodes[i]; + buffer << getActionsDAGNodeName(function_argument_node.get()); + + if (i + 1 != function_arguments_nodes_size) + buffer << ", "; + } + + buffer << ')'; + + result = buffer.str(); + break; + } + case QueryTreeNodeType::QUERY: + { + auto query_hash = node->getTreeHash(); + + result = "__subquery_" + std::to_string(query_hash.first) + '_' + std::to_string(query_hash.second); + break; + } + case QueryTreeNodeType::LAMBDA: + { + auto lambda_hash = node->getTreeHash(); + + result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + break; + } + default: + { + result = node->getName(); + break; + } + } + + return result; + } + std::vector actions_stack; + const PlannerContext & planner_context; }; class CollectSourceColumnsMatcher @@ -471,7 +981,7 @@ public: struct Data { - NameSet source_columns_set; + PlannerContext & planner_context; }; static void visit(QueryTreeNodePtr & node, Data & data) @@ -480,29 +990,803 @@ public: if (!column_node) return; - if (column_node->getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA) + auto column_source_node = column_node->getColumnSource(); + auto column_source_node_type = column_source_node->getNodeType(); + + if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "ARRAY JOIN is not supported"); + + if (column_source_node_type == QueryTreeNodeType::LAMBDA) return; - /// Replace ALIAS column with expression - - if (column_node->hasAliasExpression()) + if (column_node->hasExpression()) { - node = column_node->getAliasExpression(); + /// Replace ALIAS column with expression + node = column_node->getExpression(); visit(node, data); return; } - data.source_columns_set.insert(column_node->getColumnName()); + if (column_source_node_type != QueryTreeNodeType::TABLE && + column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && + column_source_node_type != QueryTreeNodeType::QUERY) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected table, table function or query column source. Actual {}", + column_source_node->formatASTForErrorMessage()); + + auto & table_expression_node_to_columns = data.planner_context.table_expression_node_to_columns; + auto & table_expression_column_node_to_column_identifier = data.planner_context.table_expression_column_node_to_column_identifier; + + auto [it, _] = table_expression_node_to_columns.emplace(column_source_node.get(), TableExpressionColumns()); + auto [source_columns_set_it, inserted] = it->second.source_columns_set.insert(column_node->getColumnName()); + + if (inserted) + { + auto column_identifier = data.planner_context.getColumnUniqueIdentifier(); + table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier); + it->second.column_name_to_column_identifier.emplace(column_node->getColumnName(), column_identifier); + it->second.source_input_columns.emplace_back(column_node->getColumn()); + } + else + { + auto column_identifier_it = it->second.column_name_to_column_identifier.find(column_node->getColumnName()); + if (column_identifier_it == it->second.column_name_to_column_identifier.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column node {} column identifier is not initialized", + column_node->formatASTForErrorMessage()); + + table_expression_column_node_to_column_identifier.emplace(column_node, column_identifier_it->second); + } } - static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) + static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { - return true; + return child_node->getNodeType() != QueryTreeNodeType::QUERY; } }; using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor; +ActionsDAGPtr convertExpressionNodeIntoDAG(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & inputs, const PlannerContext & planner_context) +{ + ActionsDAGPtr action_dag = std::make_shared(inputs); + QueryTreeActionsVisitor actions_visitor(action_dag, planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(expression_node); + action_dag->getOutputs().clear(); + + for (auto & expression_dag_index_node : expression_dag_index_nodes) + action_dag->getOutputs().push_back(expression_dag_index_node); + + return action_dag; +} + +struct JoinTreeNodePlan +{ + QueryPlanBuilder plan_builder; + std::vector actions_chain_node_indices; +}; + +JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context); + +JoinTreeNodePlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, + SelectQueryInfo & table_expression_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + + QueryPlan query_plan; + + /** Use default columns to support case when there are no columns in query. + * Example: SELECT 1; + */ + const auto & [it, _] = planner_context.table_expression_node_to_columns.emplace(table_expression.get(), TableExpressionColumns()); + auto & table_expression_columns = it->second; + + if (table_node || table_function_node) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + + auto from_stage = storage->getQueryProcessingStage(planner_context.query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); + + Names column_names(table_expression_columns.source_columns_set.begin(), table_expression_columns.source_columns_set.end()); + + std::optional read_additional_column; + + bool plan_has_multiple_table_expressions = planner_context.table_expression_node_to_columns.size() > 1; + if (column_names.empty() && (plan_has_multiple_table_expressions || storage->getName() == "SystemOne")) + { + auto column_names_and_types = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns()); + read_additional_column = column_names_and_types.front(); + } + + if (read_additional_column) + { + column_names.push_back(read_additional_column->name); + table_expression_columns.source_columns_set.emplace(read_additional_column->name); + table_expression_columns.source_input_columns.emplace_back(*read_additional_column); + table_expression_columns.column_name_to_column_identifier.emplace(read_additional_column->name, planner_context.getColumnUniqueIdentifier()); + } + + if (!column_names.empty()) + { + size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; + size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + storage->read(query_plan, column_names, storage_snapshot, table_expression_query_info, planner_context.query_context, from_stage, max_block_size, max_streams); + } + + /// Create step which reads from empty source if storage has no data. + if (!query_plan.isInitialized()) + { + auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + } + else if (query_node) + { + InterpreterSelectQueryAnalyzer interpeter(table_expression, select_query_options, planner_context.query_context); + interpeter.initializeQueryPlanIfNeeded(); + query_plan = std::move(interpeter).extractQueryPlan(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function or query. Actual {}", table_expression->formatASTForErrorMessage()); + } + + auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + + for (const auto & [column_name, column_identifier] : table_expression_columns.column_name_to_column_identifier) + { + auto position = query_plan.getCurrentDataStream().header.getPositionByName(column_name); + const auto * node_to_rename = rename_actions_dag->getOutputs()[position]; + rename_actions_dag->getOutputs()[position] = &rename_actions_dag->addAlias(*node_to_rename, column_identifier); + } + + planner_context.actions_chain.addNode(std::make_unique(rename_actions_dag, true /*available_output_columns_only_aliases*/)); + size_t actions_chain_node_index = planner_context.actions_chain.getLastNodeIndex(); + + QueryPlanBuilder builder(std::move(query_plan)); + + builder.addBuildStep([rename_actions_dag](QueryPlan & build_plan) { + auto rename_step = std::make_unique(build_plan.getCurrentDataStream(), rename_actions_dag); + rename_step->setStepDescription("Change column names to column identifiers"); + build_plan.addStep(std::move(rename_step)); + }); + + return {std::move(builder), {actions_chain_node_index}}; +} + +class JoinClause +{ +public: + void addKey(const ActionsDAG::Node * left_key_node, const ActionsDAG::Node * right_key_node) + { + left_key_nodes.emplace_back(left_key_node); + right_key_nodes.emplace_back(right_key_node); + } + + void addCondition(JoinTableSide table_side, const ActionsDAG::Node * condition_node) + { + auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; + filter_condition_nodes.push_back(condition_node); + } + + const ActionsDAG::NodeRawConstPtrs & getLeftKeyNodes() const + { + return left_key_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getRightKeyNodes() const + { + return right_key_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getLeftFilterConditionNodes() const + { + return left_filter_condition_nodes; + } + + const ActionsDAG::NodeRawConstPtrs & getRightFilterConditionNodes() const + { + return right_filter_condition_nodes; + } + + void clearConditionNodes(JoinTableSide table_side) + { + auto & filter_condition_nodes = table_side == JoinTableSide::Left ? left_filter_condition_nodes : right_filter_condition_nodes; + filter_condition_nodes.clear(); + } + + void dump(WriteBuffer & buffer) const + { + auto dump_dag_nodes = [&](const ActionsDAG::NodeRawConstPtrs & dag_nodes) + { + String dag_nodes_dump; + + if (!dag_nodes.empty()) + { + for (const auto & dag_node : dag_nodes) + { + dag_nodes_dump += dag_node->result_name; + dag_nodes_dump += ", "; + } + + dag_nodes_dump.pop_back(); + dag_nodes_dump.pop_back(); + } + + return dag_nodes_dump; + }; + + buffer << "left_key_nodes: " << dump_dag_nodes(left_key_nodes); + buffer << " right_key_nodes: " << dump_dag_nodes(right_key_nodes); + + if (!left_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(left_filter_condition_nodes); + + if (!right_filter_condition_nodes.empty()) + buffer << " left_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); + } + + [[maybe_unused]] String dump() const + { + WriteBufferFromOwnString buffer; + dump(buffer); + + return buffer.str(); + } +private: + ActionsDAG::NodeRawConstPtrs left_key_nodes; + ActionsDAG::NodeRawConstPtrs right_key_nodes; + + ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; + ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; +}; + +using JoinClauses = std::vector; + +std::optional extractJoinTableSideFromExpression(const ActionsDAG::Node * expression_root_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node) +{ + std::optional table_side; + std::vector nodes_to_process; + nodes_to_process.push_back(expression_root_node); + + 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); + + if (node_to_process->type != ActionsDAG::ActionType::INPUT) + continue; + + const auto & input_name = node_to_process->result_name; + + 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, ", ")); + + auto input_table_side = left_table_expression_contains_input ? 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", + join_node.formatASTForErrorMessage()); + + table_side = input_table_side; + } + + return table_side; +} + +void buildJoinClause(ActionsDAGPtr join_expression_dag, + const ActionsDAG::Node * join_expressions_actions_node, + const NameSet & left_table_expression_columns_names, + const NameSet & right_table_expression_columns_names, + const JoinNode & join_node, + JoinClause & join_clause) +{ + /// For and function go into children + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "and") + { + for (const auto & child : join_expressions_actions_node->children) + { + buildJoinClause(join_expression_dag, + child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node, + join_clause); + } + + return; + } + + if (join_expressions_actions_node->function && join_expressions_actions_node->function->getName() == "equals") + { + const auto * equals_left_child = join_expressions_actions_node->children.at(0); + const auto * equals_right_child = join_expressions_actions_node->children.at(1); + + auto left_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_left_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + auto right_equals_expression_side_optional = extractJoinTableSideFromExpression(equals_right_child, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!left_equals_expression_side_optional && !right_equals_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->function->getName()); + } + else if (left_equals_expression_side_optional && !right_equals_expression_side_optional) + { + join_clause.addCondition(*left_equals_expression_side_optional, join_expressions_actions_node); + } + else if (!left_equals_expression_side_optional && right_equals_expression_side_optional) + { + join_clause.addCondition(*right_equals_expression_side_optional, join_expressions_actions_node); + } + else + { + auto left_equals_expression_side = *left_equals_expression_side_optional; + auto right_equals_expression_side = *right_equals_expression_side_optional; + + if (left_equals_expression_side != right_equals_expression_side) + join_clause.addKey(equals_left_child, equals_right_child); + else + join_clause.addCondition(left_equals_expression_side, join_expressions_actions_node); + } + + return; + } + + auto expression_side_optional = extractJoinTableSideFromExpression(join_expressions_actions_node, + left_table_expression_columns_names, + right_table_expression_columns_names, + join_node); + + if (!expression_side_optional) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} with constants is not supported", + join_node.formatASTForErrorMessage()); + + auto expression_side = *expression_side_optional; + + join_clause.addCondition(expression_side, join_expressions_actions_node); +} + +struct JoinClausesAndActions +{ + JoinClauses join_clauses; + ActionsDAGPtr join_expression_actions; + ActionsDAGPtr left_join_expressions_actions; + ActionsDAGPtr right_join_expressions_actions; +}; + +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 PlannerContext & planner_context) +{ + std::cout << "buildJoinClausesAndActions " << join_node.formatASTForErrorMessage() << std::endl; + + ActionsDAGPtr join_expression_actions = std::make_shared(join_expression_input_columns); + // std::cout << "buildJoinClausesAndActions join expression actions dag before visitor " << std::endl; + // std::cout << join_expression_actions->dumpDAG() << std::endl; + + QueryTreeActionsVisitor join_expression_visitor(join_expression_actions, planner_context); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_node.getJoinExpression()); + if (join_expression_dag_node_raw_pointers.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} ON clause contains multiple expressions", + join_node.formatASTForErrorMessage()); + + // std::cout << "buildJoinClausesAndActions join expression actions dag after visitor " << std::endl; + // std::cout << join_expression_actions->dumpDAG() << std::endl; + + 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()); + + std::cout << "buildJoinClausesAndActions join expressions actions DAG dump " << std::endl; + std::cout << join_expression_actions->dumpDAG() << std::endl; + + std::cout << "root node " << join_expressions_actions_root_node << std::endl; + + size_t left_table_expression_columns_size = left_table_expression_columns.size(); + + Names join_left_actions_names; + join_left_actions_names.reserve(left_table_expression_columns_size); + + NameSet join_left_actions_names_set; + join_left_actions_names_set.reserve(left_table_expression_columns_size); + + for (const auto & left_table_expression_column : left_table_expression_columns) + { + join_left_actions_names.push_back(left_table_expression_column.name); + join_left_actions_names_set.insert(left_table_expression_column.name); + } + + size_t right_table_expression_columns_size = right_table_expression_columns.size(); + + Names join_right_actions_names; + join_right_actions_names.reserve(right_table_expression_columns_size); + + NameSet join_right_actions_names_set; + join_right_actions_names_set.reserve(right_table_expression_columns_size); + + for (const auto & right_table_expression_column : right_table_expression_columns) + { + join_right_actions_names.push_back(right_table_expression_column.name); + join_right_actions_names_set.insert(right_table_expression_column.name); + } + + JoinClausesAndActions result; + result.join_expression_actions = join_expression_actions; + + const auto & function_name = join_expressions_actions_root_node->function->getName(); + if (function_name == "or") + { + for (const auto & child : join_expressions_actions_root_node->children) + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + child, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + } + else + { + result.join_clauses.emplace_back(); + + buildJoinClause(join_expression_actions, + join_expressions_actions_root_node, + join_left_actions_names_set, + join_right_actions_names_set, + join_node, + result.join_clauses.back()); + } + + auto and_function = FunctionFactory::instance().get("and", planner_context.query_context); + + auto add_necessary_name_if_needed = [&](JoinTableSide join_table_side, const String & name) + { + auto & necessary_names = join_table_side == JoinTableSide::Left ? join_left_actions_names : join_right_actions_names; + auto & necessary_names_set = join_table_side == JoinTableSide::Left ? join_left_actions_names_set : join_right_actions_names_set; + + auto [_, inserted] = necessary_names_set.emplace(name); + if (inserted) + necessary_names.push_back(name); + }; + + for (auto & join_clause : result.join_clauses) + { + const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!left_filter_condition_nodes.empty()) + { + 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, {}); + else + dag_filter_condition_node = left_filter_condition_nodes[0]; + + join_clause.clearConditionNodes(JoinTableSide::Left); + join_clause.addCondition(JoinTableSide::Left, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); + } + + const auto & right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!right_filter_condition_nodes.empty()) + { + 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, {}); + else + dag_filter_condition_node = right_filter_condition_nodes[0]; + + join_clause.clearConditionNodes(JoinTableSide::Right); + join_clause.addCondition(JoinTableSide::Right, dag_filter_condition_node); + + join_expression_actions->addOrReplaceInOutputs(*dag_filter_condition_node); + + add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); + } + + for (const auto & left_key_node : join_clause.getLeftKeyNodes()) + { + join_expression_actions->addOrReplaceInOutputs(*left_key_node); + add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); + } + + for (const auto & right_key_node : join_clause.getRightKeyNodes()) + { + join_expression_actions->addOrReplaceInOutputs(*right_key_node); + 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->removeUnusedActions(join_left_actions_names); + + result.right_join_expressions_actions = join_expression_actions->clone(); + result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); + + return result; +} + +JoinTreeNodePlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto & join_node = join_tree_node->as(); + + auto left_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getLeftTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto left_plan_builder = std::move(left_plan_build_result.plan_builder); + ColumnsWithTypeAndName left_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(left_plan_build_result.actions_chain_node_indices); + + auto right_plan_build_result = buildQueryPlanForJoinTreeNode(join_node.getRightTableExpression(), + select_query_info, + select_query_options, + planner_context); + auto right_plan_builder = std::move(right_plan_build_result.plan_builder); + auto right_plan_output_columns = planner_context.actions_chain.getAvailableOutputColumns(right_plan_build_result.actions_chain_node_indices); + + if (join_node.getStrictness() == JoinStrictness::Asof) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} ASOF is not supported", + join_node.formatASTForErrorMessage()); + + JoinClausesAndActions join_clauses_and_actions; + + std::vector actions_chain_node_indices; + std::vector actions_chain_right_plan_node_indexes; + + if (join_node.getJoinExpression()) + { + if (join_node.isUsingJoinExpression()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "JOIN {} USING is unsupported", + join_node.formatASTForErrorMessage()); + + auto join_expression_input_columns = left_plan_output_columns; + join_expression_input_columns.insert(join_expression_input_columns.end(), right_plan_output_columns.begin(), right_plan_output_columns.end()); + join_clauses_and_actions = buildJoinClausesAndActions(join_expression_input_columns, + left_plan_output_columns, + right_plan_output_columns, + join_node, + planner_context); + + auto left_join_actions_node = std::make_unique(join_clauses_and_actions.left_join_expressions_actions); + left_join_actions_node->addParentIndices(left_plan_build_result.actions_chain_node_indices); + planner_context.actions_chain.addNode(std::move(left_join_actions_node)); + actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); + + auto right_join_actions_node = std::make_unique(join_clauses_and_actions.right_join_expressions_actions); + right_join_actions_node->addParentIndices(right_plan_build_result.actions_chain_node_indices); + planner_context.actions_chain.addNode(std::move(right_join_actions_node)); + actions_chain_node_indices.push_back(planner_context.actions_chain.getLastNodeIndex()); + actions_chain_right_plan_node_indexes.push_back(planner_context.actions_chain.getLastNodeIndex()); + + left_plan_builder.addBuildStep([left_join_expressions_actions = join_clauses_and_actions.left_join_expressions_actions](QueryPlan & build_plan) + { + auto left_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), left_join_expressions_actions); + left_join_expressions_actions_step->setStepDescription("Join actions"); + build_plan.addStep(std::move(left_join_expressions_actions_step)); + }); + + right_plan_builder.addBuildStep([right_join_expressions_actions = join_clauses_and_actions.right_join_expressions_actions](QueryPlan & build_plan) + { + auto right_join_expressions_actions_step = std::make_unique(build_plan.getCurrentDataStream(), right_join_expressions_actions); + right_join_expressions_actions_step->setStepDescription("Join actions"); + build_plan.addStep(std::move(right_join_expressions_actions_step)); + }); + } + else + { + actions_chain_right_plan_node_indexes = right_plan_build_result.actions_chain_node_indices; + actions_chain_node_indices.insert(actions_chain_node_indices.end(), actions_chain_right_plan_node_indexes.begin(), actions_chain_right_plan_node_indexes.end()); + } + + std::vector builders; + builders.emplace_back(std::move(left_plan_builder)); + builders.emplace_back(std::move(right_plan_builder)); + + QueryPlanBuilder builder(std::move(builders), [join_clauses_and_actions, actions_chain_right_plan_node_indexes, &join_node, &planner_context](std::vector build_query_plans) + { + if (build_query_plans.size() != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Join step expects 2 query plans. Actual {}", build_query_plans.size()); + + auto left_plan = std::move(build_query_plans[0]); + auto right_plan = std::move(build_query_plans[1]); + + auto table_join = std::make_shared(); + table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + if (join_node.getKind() == JoinKind::Comma) + table_join->getTableJoin().kind = JoinKind::Cross; + table_join->getTableJoin().strictness = JoinStrictness::All; + + NameSet join_clauses_right_column_names; + + if (join_node.getJoinExpression()) + { + const auto & join_clauses = join_clauses_and_actions.join_clauses; + auto & table_join_clauses = table_join->getClauses(); + + for (const auto & join_clause : join_clauses) + { + table_join_clauses.emplace_back(); + auto & table_join_clause = table_join_clauses.back(); + + const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); + const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); + + size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); + assert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + + for (size_t i = 0; i < join_clause_key_nodes_size; ++i) + { + table_join_clause.key_names_left.push_back(join_clause_left_key_nodes[i]->result_name); + table_join_clause.key_names_right.push_back(join_clause_right_key_nodes[i]->result_name); + join_clauses_right_column_names.insert(join_clause_right_key_nodes[i]->result_name); + } + + const auto & join_clause_get_left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); + if (!join_clause_get_left_filter_condition_nodes.empty()) + { + if (join_clause_get_left_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} left filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_left_filter_condition_nodes.size()); + + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } + + const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); + if (!join_clause_get_right_filter_condition_nodes.empty()) + { + if (join_clause_get_right_filter_condition_nodes.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "JOIN {} right filter conditions size must be 1. Actual {}", + join_node.formatASTForErrorMessage(), + join_clause_get_right_filter_condition_nodes.size()); + + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + join_clauses_right_column_names.insert(join_clause_right_filter_condition_name); + } + } + } + + auto left_table_names = left_plan.getCurrentDataStream().header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentDataStream().header.getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + NamesAndTypesList columns_added_by_join; + for (auto & column_from_joined_table : columns_from_joined_table) + { + for (const auto & actions_chain_right_plan_node_index : actions_chain_right_plan_node_indexes) + { + const auto & child_required_ouput_columns_names = planner_context.actions_chain[actions_chain_right_plan_node_index]->getChildRequiredOutputColumnsNames(); + + if (child_required_ouput_columns_names.contains(column_from_joined_table.name)) + { + columns_added_by_join.insert(columns_added_by_join.end(), column_from_joined_table); + break; + } + } + } + + table_join->setColumnsAddedByJoin(columns_added_by_join); + + size_t max_block_size = planner_context.query_context->getSettingsRef().max_block_size; + size_t max_streams = planner_context.query_context->getSettingsRef().max_threads; + + JoinPtr join_ptr = std::make_shared(table_join, right_plan.getCurrentDataStream().header, false /*any_take_last_row*/); + QueryPlanStepPtr join_step = std::make_unique( + left_plan.getCurrentDataStream(), + right_plan.getCurrentDataStream(), + join_ptr, + max_block_size, + max_streams, + false /*optimize_read_in_order*/); + + join_step->setStepDescription(fmt::format("JOIN {}", JoinPipelineType::FillRightFirst)); + + std::vector plans; + plans.emplace_back(std::make_unique(std::move(left_plan))); + plans.emplace_back(std::make_unique(std::move(right_plan))); + + auto result = QueryPlan(); + result.unitePlans(std::move(join_step), {std::move(plans)}); + + return result; + }); + + return {std::move(builder), actions_chain_node_indices}; +} + +JoinTreeNodePlan buildQueryPlanForJoinTreeNode(QueryTreeNodePtr join_tree_node, + SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContext & planner_context) +{ + auto join_tree_node_type = join_tree_node->getNodeType(); + + switch (join_tree_node_type) + { + case QueryTreeNodeType::QUERY: + [[fallthrough]]; + case QueryTreeNodeType::TABLE: + [[fallthrough]]; + case QueryTreeNodeType::TABLE_FUNCTION: + { + SelectQueryInfo table_expression_query_info = select_query_info; + return buildQueryPlanForTableExpression(join_tree_node, table_expression_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::JOIN: + { + return buildQueryPlanForJoinNode(join_tree_node, select_query_info, select_query_options, planner_context); + } + case QueryTreeNodeType::ARRAY_JOIN: + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ARRAY JOIN is not supported"); + } + default: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected query, table, table function, join or array join query node. Actual {}", join_tree_node->formatASTForErrorMessage()); + } + } +} + +} + InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( const ASTPtr & query_, const SelectQueryOptions & select_query_options_, @@ -586,124 +1870,102 @@ void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded() if (query_plan.isInitialized()) return; - auto & query_tree_typed = query_tree->as(); + auto & query_node = query_tree->as(); - ActionsDAGPtr action_dag = std::make_shared(); - ColumnsWithTypeAndName inputs; + auto current_context = getContext(); - CollectSourceColumnsVisitor::Data data; + SelectQueryInfo select_query_info; + select_query_info.original_query = query; + select_query_info.query = query; + + PlannerContext planner_context; + planner_context.query_context = getContext(); + + CollectSourceColumnsVisitor::Data data {planner_context}; CollectSourceColumnsVisitor collect_source_columns_visitor(data); collect_source_columns_visitor.visit(query_tree); - NameSet source_columns_set = std::move(data.source_columns_set); + JoinTreeNodePlan join_tree_node_plan = buildQueryPlanForJoinTreeNode(query_node.getFrom(), select_query_info, select_query_options, planner_context); + auto query_plan_builder = std::move(join_tree_node_plan.plan_builder); + auto action_chain_node_parent_indices = join_tree_node_plan.actions_chain_node_indices; - // std::cout << "DAG before " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; + if (query_node.hasWhere()) + { + ColumnsWithTypeAndName where_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + planner_context.where_actions = convertExpressionNodeIntoDAG(query_node.getWhere(), where_input, planner_context); + planner_context.where_action_node_name = planner_context.where_actions->getOutputs().at(0)->result_name; - QueryTreeActionsVisitor visitor(action_dag, getContext()); - auto projection_action_dag_nodes = visitor.visit(query_tree_typed.getProjectionNode()); + auto where_actions_node = std::make_unique(planner_context.where_actions); + where_actions_node->addParentIndices(action_chain_node_parent_indices); + planner_context.actions_chain.addNode(std::move(where_actions_node)); + action_chain_node_parent_indices = {planner_context.actions_chain.getLastNodeIndex()}; + + size_t where_node_index = planner_context.actions_chain.size(); + + query_plan_builder.addBuildStep([&, where_node_index](QueryPlan & build_plan) + { + bool remove_filter = !planner_context.actions_chain.at(where_node_index)->getChildRequiredOutputColumnsNames().contains(planner_context.where_action_node_name); + auto where_step = std::make_unique(build_plan.getCurrentDataStream(), + planner_context.where_actions, + planner_context.where_action_node_name, + remove_filter); + where_step->setStepDescription("WHERE"); + build_plan.addStep(std::move(where_step)); + }); + } + + ColumnsWithTypeAndName projection_input = planner_context.actions_chain.getAvailableOutputColumns(action_chain_node_parent_indices); + planner_context.projection_actions = convertExpressionNodeIntoDAG(query_node.getProjectionNode(), projection_input, planner_context); + + auto projection_actions_node = std::make_unique(planner_context.projection_actions); + projection_actions_node->addParentIndices(action_chain_node_parent_indices); + planner_context.actions_chain.addNode(std::move(projection_actions_node)); + + const auto & projection_action_dag_nodes = planner_context.projection_actions->getOutputs(); size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size(); - // std::cout << "Projection action dag nodes size " << projection_action_dag_nodes_size << std::endl; - // for (size_t i = 0; i < projection_action_dag_nodes_size; ++i) - // { - // std::cout << "DAG node " << projection_action_dag_nodes[i] << std::endl; - // } - - // std::cout << "DAG after " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; - - auto & projection_nodes = query_tree_typed.getProjection().getNodes(); + auto & projection_nodes = query_node.getProjection().getNodes(); size_t projection_nodes_size = projection_nodes.size(); if (projection_nodes_size != projection_action_dag_nodes_size) - throw Exception( - ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryTree projection nodes size mismatch. Expected {}. Actual {}", projection_action_dag_nodes_size, projection_nodes_size); NamesWithAliases projection_names; + for (size_t i = 0; i < projection_nodes_size; ++i) { auto & node = projection_nodes[i]; + auto node_name = node->getName(); const auto * action_dag_node = projection_action_dag_nodes[i]; - auto action_dag_node_name = action_dag_node->result_name; - - action_dag->getIndex().push_back(action_dag_node); + const auto & actions_dag_node_name = action_dag_node->result_name; if (node->hasAlias()) - projection_names.push_back({action_dag_node_name, node->getAlias()}); + projection_names.push_back({actions_dag_node_name, node->getAlias()}); else - projection_names.push_back({action_dag_node_name, action_dag_node_name}); + projection_names.push_back({actions_dag_node_name, node_name}); } - action_dag->project(projection_names); + planner_context.projection_actions->project(projection_names); - // std::cout << "Final DAG " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl; - // std::cout << action_dag->dumpDAG() << std::endl; - // std::cout << "Names " << action_dag->dumpNames() << std::endl; - // std::cout << "Final DAG nodes " << std::endl; - // for (const auto & node : action_dag->getNodes()) - // { - // std::cout << "Node " << &node << " result name " << node.result_name << std::endl; - // } - - // std::cout << "Source columns " << source_columns_set.size() << std::endl; - // for (const auto & source_column : source_columns_set) - // std::cout << source_column << std::endl; - - auto current_context = getContext(); - size_t max_block_size = current_context->getSettingsRef().max_block_size; - size_t max_streams = current_context->getSettingsRef().max_threads; - - SelectQueryInfo query_info; - query_info.original_query = query; - query_info.query = query; - - auto * table_node = query_tree_typed.getFrom()->as(); - auto * table_function_node = query_tree_typed.getFrom()->as(); - auto * query_node = query_tree_typed.getFrom()->as(); - - if (table_node || table_function_node) + query_plan_builder.addBuildStep([&](QueryPlan & build_plan) { - const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto projection_step = std::make_unique(build_plan.getCurrentDataStream(), planner_context.projection_actions); + projection_step->setStepDescription("Projection"); + build_plan.addStep(std::move(projection_step)); + }); - auto from_stage = storage->getQueryProcessingStage(current_context, select_query_options.to_stage, storage_snapshot, query_info); + // std::cout << "Chain dump before finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; - Names column_names(source_columns_set.begin(), source_columns_set.end()); + planner_context.actions_chain.finalize(); - if (column_names.empty() && storage->getName() == "SystemOne") - column_names.push_back("dummy"); + // std::cout << "Chain dump after finalize" << std::endl; + // std::cout << planner_context.actions_chain.dump() << std::endl; - if (!column_names.empty()) - storage->read(query_plan, column_names, storage_snapshot, query_info, getContext(), from_stage, max_block_size, max_streams); - - /// Create step which reads from empty source if storage has no data. - if (!query_plan.isInitialized()) - { - auto source_header = storage_snapshot->getSampleBlockForColumns(column_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); - } - } - else if (query_node) - { - InterpreterSelectQueryAnalyzer interpeter(query_tree_typed.getFrom(), select_query_options, getContext()); - interpeter.initializeQueryPlanIfNeeded(); - query_plan = std::move(interpeter.query_plan); - } - else - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table or query in FROM section are supported"); - } - - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), action_dag); - projection_step->setStepDescription("Projection"); - query_plan.addStep(std::move(projection_step)); + query_plan = std::move(query_plan_builder).buildPlan(); } } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 561d4054f4e..4ba01c26bdd 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -31,9 +31,14 @@ public: bool supportsTransactions() const override { return true; } -private: void initializeQueryPlanIfNeeded(); + QueryPlan && extractQueryPlan() && + { + return std::move(query_plan); + } + +private: ASTPtr query; QueryTreeNodePtr query_tree; QueryPlan query_plan; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 5d065e564b2..7461c53a342 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -46,51 +46,49 @@ namespace std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJoin::NameToTypeMap & source) { std::vector text; - for (const auto & [k, v] : target) + for (const auto & [key, value] : target) { - auto src_type_it = source.find(k); + auto src_type_it = source.find(key); std::string src_type_name = src_type_it != source.end() ? src_type_it->second->getName() : ""; - text.push_back(fmt::format("{} : {} -> {}", k, src_type_name, v->getName())); + text.push_back(fmt::format("{} : {} -> {}", key, src_type_name, value->getName())); } + return fmt::format("{}", fmt::join(text, ", ")); } -} - -namespace -{ - struct BothSidesTag {}; struct LeftSideTag {}; struct RightSideTag {}; -template -bool forAllKeys(OnExpr & expressions, Func callback) +template +bool forKeyNamesInJoinClauses(JoinClauses & join_clauses, Func callback) { static_assert(std::is_same_v || std::is_same_v || std::is_same_v); - for (auto & expr : expressions) + for (auto & join_clause : join_clauses) { if constexpr (std::is_same_v) - assert(expr.key_names_left.size() == expr.key_names_right.size()); + assert(join_clause.key_names_left.size() == join_clause.key_names_right.size()); - size_t sz = !std::is_same_v ? expr.key_names_left.size() : expr.key_names_right.size(); - for (size_t i = 0; i < sz; ++i) + size_t key_names_size = !std::is_same_v ? join_clause.key_names_left.size() : join_clause.key_names_right.size(); + + for (size_t i = 0; i < key_names_size; ++i) { - bool cont; + bool should_continue; if constexpr (std::is_same_v) - cont = callback(expr.key_names_left[i], expr.key_names_right[i]); - if constexpr (std::is_same_v) - cont = callback(expr.key_names_left[i]); + should_continue = callback(join_clause.key_names_left[i], join_clause.key_names_right[i]); + else if constexpr (std::is_same_v) + should_continue = callback(join_clause.key_names_left[i]); if constexpr (std::is_same_v) - cont = callback(expr.key_names_right[i]); + should_continue = callback(join_clause.key_names_right[i]); - if (!cont) + if (!should_continue) return false; } } + return true; } @@ -133,6 +131,7 @@ void TableJoin::resetCollected() void TableJoin::addUsingKey(const ASTPtr & ast) { + // std::cout << "TableJoin::addUsingKey " << ast->formatForErrorMessage() << std::endl; addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } @@ -237,6 +236,12 @@ ASTPtr TableJoin::rightKeysList() const return keys_list; } +void TableJoin::setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) +{ + columns_from_joined_table = std::move(columns_from_joined_table_value); + deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); +} + Names TableJoin::requiredJoinedNames() const { Names key_names_right = getAllNames(JoinTableSide::Right); @@ -256,16 +261,19 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { - NameSet required; - forAllKeys(clauses, [this, &required](const auto & name) + NameSet required_right_column_names; + forKeyNamesInJoinClauses(clauses, [&](const auto & right_column_name) { - auto rename = renamedRightColumnName(name); + auto renamed_right_column = renamedRightColumnName(right_column_name); + for (const auto & column : columns_added_by_join) - if (rename == column.name) - required.insert(name); + if (renamed_right_column == column.name) + required_right_column_names.insert(renamed_right_column); + return true; }); - return required; + + return required_right_column_names; } NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const @@ -286,7 +294,7 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto if (required_keys.empty()) return required_right_keys; - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (required_keys.contains(right_key_name) && !required_right_keys.has(right_key_name)) { @@ -294,8 +302,10 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto required_right_keys.insert(right_key); keys_sources.push_back(left_key_name); } + return true; }); + return required_right_keys; } @@ -311,9 +321,15 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { + // std::cout << "TableJoin::addJoinedColumn " << joined_column.dump() << std::endl; columns_added_by_join.emplace_back(joined_column); } +void TableJoin::setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) +{ + columns_added_by_join = columns_added_by_join_value; +} + NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const { NamesAndTypesList result; @@ -416,6 +432,75 @@ bool TableJoin::needStreamWithNonJoinedRows() const return isRightOrFull(kind()); } +static std::optional getDictKeyName(const String & dict_name , ContextPtr context) +{ + auto dictionary = context->getExternalDictionariesLoader().getDictionary(dict_name, context); + if (!dictionary) + return {}; + + if (const auto & structure = dictionary->getStructure(); structure.id) + return structure.id->name; + return {}; +} + +bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) +{ + bool allowed_inner = isInner(kind()) && strictness() == JoinStrictness::All; + bool allowed_left = isLeft(kind()) && (strictness() == JoinStrictness::Any || + strictness() == JoinStrictness::All || + strictness() == JoinStrictness::Semi || + strictness() == JoinStrictness::Anti); + + /// Support ALL INNER, [ANY | ALL | SEMI | ANTI] LEFT + if (!allowed_inner && !allowed_left) + return false; + + if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1) + return false; + + const auto & right_key = getOnlyClause().key_names_right[0]; + + /// TODO: support 'JOIN ... ON expr(dict_key) = table_key' + auto it_key = original_names.find(right_key); + if (it_key == original_names.end()) + return false; + + if (!right_storage_dictionary) + return false; + + auto dict_name = right_storage_dictionary->getDictionaryName(); + + auto dict_key = getDictKeyName(dict_name, context); + if (!dict_key.has_value() || *dict_key != it_key->second) + return false; /// JOIN key != Dictionary key + + Names src_names; + NamesAndTypesList dst_columns; + for (const auto & col : sample_block) + { + if (col.name == right_key) + continue; /// do not extract key column + + auto it = original_names.find(col.name); + if (it != original_names.end()) + { + String original = it->second; + src_names.push_back(original); + dst_columns.push_back({col.name, col.type}); + } + else + { + /// Can't extract column from dictionary table + /// TODO: Sometimes it should be possible to reconstruct required column, + /// e.g. if it's an expression depending on dictionary attributes + return false; + } + } + dictionary_reader = std::make_shared(dict_name, src_names, dst_columns, context); + + return true; +} + static void renameIfNeeded(String & name, const NameToNameMap & renames) { if (const auto it = renames.find(name); it != renames.end()) @@ -473,7 +558,7 @@ TableJoin::createConvertingActions( log_actions("Right", right_converting_actions); } - forAllKeys(clauses, [&](auto & left_key, auto & right_key) + forKeyNamesInJoinClauses(clauses, [&](auto & left_key, auto & right_key) { renameIfNeeded(left_key, left_key_column_rename); renameIfNeeded(right_key, right_key_column_rename); @@ -507,7 +592,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig throw DB::Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); } - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { auto ltypeit = left_types.find(left_key_name); auto rtypeit = right_types.find(right_key_name); @@ -708,7 +793,7 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const if (hasUsing()) { const auto & required_right_keys = requiredRightKeys(); - forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + forKeyNamesInJoinClauses(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (!required_right_keys.contains(right_key_name)) left_to_right_key_remap[left_key_name] = right_key_name; @@ -722,14 +807,16 @@ Names TableJoin::getAllNames(JoinTableSide side) const { Names res; auto func = [&res](const auto & name) { res.emplace_back(name); return true; }; + if (side == JoinTableSide::Left) - forAllKeys(clauses, func); + forKeyNamesInJoinClauses(clauses, func); else - forAllKeys(clauses, func); + forKeyNamesInJoinClauses(clauses, func); + return res; } -void TableJoin::assertHasOneOnExpr() const +void TableJoin::assertHasSingleClause() const { if (!oneDisjunct()) { diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 3835ef77deb..cfa05cb7de1 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -55,15 +55,26 @@ public: ASTPtr on_filter_condition_left; ASTPtr on_filter_condition_right; + std::string analyzer_left_filter_condition_column_name; + std::string analyzer_right_filter_condition_column_name; + JoinOnClause() = default; std::pair condColumnNames() const { std::pair res; + + if (!analyzer_left_filter_condition_column_name.empty()) + res.first = analyzer_left_filter_condition_column_name; + + if (!analyzer_right_filter_condition_column_name.empty()) + res.second = analyzer_right_filter_condition_column_name; + if (on_filter_condition_left) res.first = on_filter_condition_left->getColumnName(); if (on_filter_condition_right) res.second = on_filter_condition_right->getColumnName(); + return res; } @@ -111,9 +122,6 @@ private: * to the subquery will be added expression `expr(t2 columns)`. * It's possible to use name `expr(t2 columns)`. */ - - friend class TreeRewriter; - SizeLimits size_limits; const size_t default_max_bytes = 0; const bool join_use_nulls = false; @@ -124,9 +132,6 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - /// the limit has no technical reasons, it supposed to improve safety - const size_t MAX_DISJUNCTS = 16; /// NOLINT - ASTs key_asts_left; ASTs key_asts_right; @@ -170,7 +175,7 @@ private: void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr); - void assertHasOneOnExpr() const; + void assertHasSingleClause() const; /// Calculates common supertypes for corresponding join key columns. template @@ -178,6 +183,8 @@ private: NamesAndTypesList correctedColumnsAddedByJoin() const; + void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); + public: TableJoin() = default; @@ -217,8 +224,8 @@ public: bool allowParallelHashJoin() const; bool joinUseNulls() const { return join_use_nulls; } - bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(table_join.kind); } - bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(table_join.kind); } + bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(kind()); } + bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(kind()); } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } @@ -229,8 +236,11 @@ public: bool oneDisjunct() const; - JoinOnClause & getOnlyClause() { assertHasOneOnExpr(); return clauses[0]; } - const JoinOnClause & getOnlyClause() const { assertHasOneOnExpr(); return clauses[0]; } + ASTTableJoin & getTableJoin() { return table_join; } + const ASTTableJoin & getTableJoin() const { return table_join; } + + JoinOnClause & getOnlyClause() { assertHasSingleClause(); return clauses[0]; } + const JoinOnClause & getOnlyClause() const { assertHasSingleClause(); return clauses[0]; } std::vector & getClauses() { return clauses; } const std::vector & getClauses() const { return clauses; } @@ -266,13 +276,13 @@ public: NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const; NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_required_columns) const; - void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); size_t rightKeyInclusion(const String & name) const; NameSet requiredRightKeys() const; bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value); template void addJoinedColumnsAndCorrectTypesImpl(TColumns & left_columns, bool correct_nullability); @@ -294,15 +304,9 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix); const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } - - Names columnsAddedByJoin() const - { - Names res; - for (const auto & col : columns_added_by_join) - res.push_back(col.name); - return res; - } + const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } /// StorageJoin overrides key names (cause of different names qualification) void setRightKeys(const Names & keys) { getOnlyClause().key_names_right = keys; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ac49d79c6ba..da12dccd8d8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -611,7 +611,7 @@ void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const AS } } -void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, ASTTableJoin & out_table_join) +void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, std::shared_ptr & analyzed_join) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) @@ -649,7 +649,7 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul throw Exception("ANY FULL JOINs are not implemented", ErrorCodes::NOT_IMPLEMENTED); } - out_table_join = table_join; + analyzed_join->getTableJoin() = table_join; } /// Evaluate expression and return boolean value if it can be interpreted as bool. @@ -1236,14 +1236,11 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { const auto & right_table = tables_with_columns[1]; - auto & cols_from_joined = result.analyzed_join->columns_from_joined_table; - cols_from_joined = right_table.columns; + auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table - cols_from_joined.insert(cols_from_joined.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - - result.analyzed_join->deduplicateAndQualifyColumnNames( - source_columns_set, right_table.table.getQualifiedNamePrefix()); + columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); + result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); @@ -1254,7 +1251,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( NameSet all_source_columns_set = source_columns_set; if (table_join) { - for (const auto & [name, _] : table_join->columns_from_joined_table) + for (const auto & [name, _] : table_join->columnsFromJoinedTable()) all_source_columns_set.insert(name); } @@ -1304,7 +1301,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); setJoinStrictness( - *select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); + *select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join); auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as() : nullptr; if (table_join_ast && tables_with_columns.size() >= 2) diff --git a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql index 0f9efdb95cd..e03a65a4e4f 100644 --- a/tests/queries/0_stateless/02337_analyzer_columns_basic.sql +++ b/tests/queries/0_stateless/02337_analyzer_columns_basic.sql @@ -29,6 +29,7 @@ INSERT INTO test_table VALUES (0, 'Value'); SELECT 'Table access without table name qualification'; SELECT test_id FROM test_table; -- { serverError 47 } +SELECT test_id FROM test_unknown_table; -- { serverError 60 } DESCRIBE (SELECT id FROM test_table); SELECT id FROM test_table; diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.reference b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference index 8fa16a27152..7025cff6909 100644 --- a/tests/queries/0_stateless/02369_analyzer_array_join_function.reference +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.reference @@ -42,41 +42,3 @@ 0 1 1 0 2 2 0 3 3 --- -0 1 1 -0 1 2 -0 1 3 -0 1 4 -0 2 1 -0 2 2 -0 2 3 -0 2 4 -0 3 1 -0 3 2 -0 3 3 -0 3 4 --- -0 1 1 5 -0 1 1 6 -0 1 2 5 -0 1 2 6 -0 1 3 5 -0 1 3 6 -0 1 4 5 -0 1 4 6 -0 2 1 5 -0 2 1 6 -0 2 2 5 -0 2 2 6 -0 2 3 5 -0 2 3 6 -0 2 4 5 -0 2 4 6 -0 3 1 5 -0 3 1 6 -0 3 2 5 -0 3 2 6 -0 3 3 5 -0 3 3 6 -0 3 4 5 -0 3 4 6 diff --git a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql index 08b3bc3c146..db9884f9cdd 100644 --- a/tests/queries/0_stateless/02369_analyzer_array_join_function.sql +++ b/tests/queries/0_stateless/02369_analyzer_array_join_function.sql @@ -1,6 +1,5 @@ SET use_analyzer = 1; - SELECT arrayJoin([1, 2, 3]); SELECT '--'; @@ -49,12 +48,12 @@ SELECT '--'; SELECT id, arrayJoin(value_1) AS a, a FROM test_table; -SELECT '--'; +-- SELECT '--'; -SELECT id, arrayJoin(value_1), arrayJoin(value_2) FROM test_table; +-- SELECT id, arrayJoin(value_1), arrayJoin(value_2) FROM test_table; -SELECT '--'; +-- SELECT '--'; -SELECT id, arrayJoin(value_1), arrayJoin(value_2), arrayJoin([5, 6]) FROM test_table; +-- SELECT id, arrayJoin(value_1), arrayJoin(value_2), arrayJoin([5, 6]) FROM test_table; DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.reference b/tests/queries/0_stateless/02371_analyzer_join_cross.reference new file mode 100644 index 00000000000..50e43ac28d1 --- /dev/null +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.reference @@ -0,0 +1,133 @@ +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 Join_1_Value_0 0 Join_2_Value_0 1 Join_3_Value_1 +0 Join_1_Value_0 0 Join_2_Value_0 2 Join_3_Value_2 +0 Join_1_Value_0 1 Join_2_Value_1 0 Join_3_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 1 Join_3_Value_1 +0 Join_1_Value_0 1 Join_2_Value_1 2 Join_3_Value_2 +0 Join_1_Value_0 2 Join_2_Value_2 0 Join_3_Value_0 +0 Join_1_Value_0 2 Join_2_Value_2 1 Join_3_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 2 Join_3_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 Join_2_Value_0 1 Join_3_Value_1 +1 Join_1_Value_1 0 Join_2_Value_0 2 Join_3_Value_2 +1 Join_1_Value_1 1 Join_2_Value_1 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +1 Join_1_Value_1 1 Join_2_Value_1 2 Join_3_Value_2 +1 Join_1_Value_1 2 Join_2_Value_2 0 Join_3_Value_0 +1 Join_1_Value_1 2 Join_2_Value_2 1 Join_3_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 2 Join_3_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 0 Join_3_Value_0 +3 Join_1_Value_3 0 Join_2_Value_0 1 Join_3_Value_1 +3 Join_1_Value_3 0 Join_2_Value_0 2 Join_3_Value_2 +3 Join_1_Value_3 1 Join_2_Value_1 0 Join_3_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 1 Join_3_Value_1 +3 Join_1_Value_3 1 Join_2_Value_1 2 Join_3_Value_2 +3 Join_1_Value_3 2 Join_2_Value_2 0 Join_3_Value_0 +3 Join_1_Value_3 2 Join_2_Value_2 1 Join_3_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 2 Join_3_Value_2 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +0 Join_1_Value_0 0 Join_2_Value_0 1 Join_3_Value_1 +0 Join_1_Value_0 0 Join_2_Value_0 2 Join_3_Value_2 +0 Join_1_Value_0 1 Join_2_Value_1 0 Join_3_Value_0 +0 Join_1_Value_0 1 Join_2_Value_1 1 Join_3_Value_1 +0 Join_1_Value_0 1 Join_2_Value_1 2 Join_3_Value_2 +0 Join_1_Value_0 2 Join_2_Value_2 0 Join_3_Value_0 +0 Join_1_Value_0 2 Join_2_Value_2 1 Join_3_Value_1 +0 Join_1_Value_0 2 Join_2_Value_2 2 Join_3_Value_2 +1 Join_1_Value_1 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 0 Join_2_Value_0 1 Join_3_Value_1 +1 Join_1_Value_1 0 Join_2_Value_0 2 Join_3_Value_2 +1 Join_1_Value_1 1 Join_2_Value_1 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +1 Join_1_Value_1 1 Join_2_Value_1 2 Join_3_Value_2 +1 Join_1_Value_1 2 Join_2_Value_2 0 Join_3_Value_0 +1 Join_1_Value_1 2 Join_2_Value_2 1 Join_3_Value_1 +1 Join_1_Value_1 2 Join_2_Value_2 2 Join_3_Value_2 +3 Join_1_Value_3 0 Join_2_Value_0 0 Join_3_Value_0 +3 Join_1_Value_3 0 Join_2_Value_0 1 Join_3_Value_1 +3 Join_1_Value_3 0 Join_2_Value_0 2 Join_3_Value_2 +3 Join_1_Value_3 1 Join_2_Value_1 0 Join_3_Value_0 +3 Join_1_Value_3 1 Join_2_Value_1 1 Join_3_Value_1 +3 Join_1_Value_3 1 Join_2_Value_1 2 Join_3_Value_2 +3 Join_1_Value_3 2 Join_2_Value_2 0 Join_3_Value_0 +3 Join_1_Value_3 2 Join_2_Value_2 1 Join_3_Value_1 +3 Join_1_Value_3 2 Join_2_Value_2 2 Join_3_Value_2 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +0 0 Join_1_Value_0 Join_1_Value_0 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +1 1 Join_1_Value_1 Join_1_Value_1 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 0 0 Join_2_Value_0 Join_2_Value_0 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 1 1 Join_2_Value_1 Join_2_Value_1 2 2 Join_3_Value_2 Join_3_Value_2 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 0 0 Join_3_Value_0 Join_3_Value_0 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 1 1 Join_3_Value_1 Join_3_Value_1 +3 3 Join_1_Value_3 Join_1_Value_3 2 2 Join_2_Value_2 Join_2_Value_2 2 2 Join_3_Value_2 Join_3_Value_2 diff --git a/tests/queries/0_stateless/02371_analyzer_join_cross.sql b/tests/queries/0_stateless/02371_analyzer_join_cross.sql new file mode 100644 index 00000000000..b7d29049398 --- /dev/null +++ b/tests/queries/0_stateless/02371_analyzer_join_cross.sql @@ -0,0 +1,78 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (3, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (2, 'Join_2_Value_2'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (2, 'Join_3_Value_2'); + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1, test_table_join_2; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value FROM test_table_join_1 AS t1, test_table_join_2 AS t2; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1, test_table_join_2, test_table_join_3; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1, test_table_join_2 AS t2, test_table_join_3 AS t3; + +SELECT id FROM test_table_join_1, test_table_join_2; -- { serverError 36 } + +SELECT value FROM test_table_join_1, test_table_join_2; -- { serverError 36 } + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3; diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.reference b/tests/queries/0_stateless/02372_analyzer_join_inner.reference new file mode 100644 index 00000000000..3f8435c35d4 --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join_inner.reference @@ -0,0 +1,51 @@ +Join without ON conditions +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +1 Join_1_Value_1 1 Join_2_Value_1 1 Join_3_Value_1 +-- +0 0 Join_1_Value_0 Join_1_Value_0 0 0 Join_2_Value_0 Join_2_Value_0 0 0 Join_3_Value_0 Join_3_Value_0 +1 1 Join_1_Value_1 Join_1_Value_1 1 1 Join_2_Value_1 Join_2_Value_1 1 1 Join_3_Value_1 Join_3_Value_1 +Join with ON conditions +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +-- +0 Join_1_Value_0 0 Join_2_Value_0 0 Join_3_Value_0 +Join only join expression use keys +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 +-- +Join_1_Value_0 Join_2_Value_0 Join_2_Value_0 +Join_1_Value_1 Join_2_Value_1 Join_2_Value_1 diff --git a/tests/queries/0_stateless/02372_analyzer_join_inner.sql b/tests/queries/0_stateless/02372_analyzer_join_inner.sql new file mode 100644 index 00000000000..5c915deef56 --- /dev/null +++ b/tests/queries/0_stateless/02372_analyzer_join_inner.sql @@ -0,0 +1,156 @@ +SET use_analyzer = 1; + +DROP TABLE IF EXISTS test_table_join_1; +CREATE TABLE test_table_join_1 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_2; +CREATE TABLE test_table_join_2 +( + id UInt64, + value String +) ENGINE = TinyLog; + +DROP TABLE IF EXISTS test_table_join_3; +CREATE TABLE test_table_join_3 +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table_join_1 VALUES (0, 'Join_1_Value_0'); +INSERT INTO test_table_join_1 VALUES (1, 'Join_1_Value_1'); +INSERT INTO test_table_join_1 VALUES (3, 'Join_1_Value_3'); + +INSERT INTO test_table_join_2 VALUES (0, 'Join_2_Value_0'); +INSERT INTO test_table_join_2 VALUES (1, 'Join_2_Value_1'); +INSERT INTO test_table_join_2 VALUES (2, 'Join_2_Value_2'); + +INSERT INTO test_table_join_3 VALUES (0, 'Join_3_Value_0'); +INSERT INTO test_table_join_3 VALUES (1, 'Join_3_Value_1'); +INSERT INTO test_table_join_3 VALUES (2, 'Join_3_Value_2'); + +SELECT 'Join without ON conditions'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON test_table_join_1.id = test_table_join_2.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT test_table_join_1.id, test_table_join_1.value, test_table_join_2.id, test_table_join_2.value, test_table_join_3.id, test_table_join_3.value +FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id + INNER JOIN test_table_join_3 ON test_table_join_2.id = test_table_join_3.id; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.id, test_table_join_1.id, t1.value, test_table_join_1.value, t2.id, test_table_join_2.id, t2.value, test_table_join_2.value, +t3.id, test_table_join_3.id, t3.value, test_table_join_3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id + INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id; + +SELECT id FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT value FROM test_table_join_1 INNER JOIN test_table_join_2 ON test_table_join_1.id = test_table_join_2.id; -- { serverError 36 } + +SELECT 'Join with ON conditions'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t1.value = 'Join_1_Value_0' AND t2.value = 'Join_2_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t2.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id AND t1.value == 'Join_1_Value_0' AND t2.value == 'Join_2_Value_0' +INNER JOIN test_table_join_3 AS t3 ON t1.id = t3.id AND t2.value == 'Join_2_Value_0' AND t3.value == 'Join_3_Value_0'; + +SELECT '--'; + +SELECT t1.id, t1.value, t2.id, t2.value, t3.id, t3.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON toString(t1.id) = toString(t2.id) AND t2.value == 'Join_2_Value_0' + INNER JOIN test_table_join_3 AS t3 ON toString(t2.id) = toString(t3.id) AND t3.value == 'Join_3_Value_0'; + +SELECT 'Join only join expression use keys'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_2 AS t3 ON t2.id = t3.id; + +SELECT '--'; + +SELECT t1.value, t2.value, t3.value +FROM test_table_join_1 AS t1 +INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id +INNER JOIN test_table_join_2 AS t3 ON t1.id = t3.id; + + +DROP TABLE test_table_join_1; +DROP TABLE test_table_join_2; +DROP TABLE test_table_join_3;