mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Added support for JOIN, ARRAY JOIN
This commit is contained in:
parent
1717d16fa4
commit
40cb904a3d
76
src/Analyzer/ArrayJoinNode.cpp
Normal file
76
src/Analyzer/ArrayJoinNode.cpp
Normal file
@ -0,0 +1,76 @@
|
||||
#include <Analyzer/ArrayJoinNode.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
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<const ArrayJoinNode &>(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<ASTArrayJoin>();
|
||||
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<ASTTablesInSelectQuery>();
|
||||
addTableExpressionIntoTablesInSelectQuery(tables_in_select_query_ast, children[table_expression_child_index]);
|
||||
|
||||
auto array_join_query_element_ast = std::make_shared<ASTTablesInSelectQueryElement>();
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
105
src/Analyzer/ArrayJoinNode.h
Normal file
105
src/Analyzer/ArrayJoinNode.h
Normal file
@ -0,0 +1,105 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/ListNode.h>
|
||||
|
||||
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<ArrayJoinNode>;
|
||||
|
||||
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<const ListNode &>();
|
||||
}
|
||||
|
||||
/// Get join expressions
|
||||
ListNode & getJoinExpressions()
|
||||
{
|
||||
return children[join_expressions_child_index]->as<ListNode &>();
|
||||
}
|
||||
|
||||
/// 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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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<const ColumnNode &>(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<ColumnNode>(column, column_source);
|
||||
return std::make_shared<ColumnNode>(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<String> parts = {column_name_qualification, column.name};
|
||||
return std::make_shared<ASTIdentifier>(std::move(parts));
|
||||
}
|
||||
|
||||
return std::make_shared<ASTIdentifier>(column.name);
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -38,6 +38,8 @@ enum class QueryTreeNodeType
|
||||
TABLE,
|
||||
TABLE_FUNCTION,
|
||||
QUERY,
|
||||
ARRAY_JOIN,
|
||||
JOIN
|
||||
};
|
||||
|
||||
/// Convert query tree node type to string
|
||||
|
159
src/Analyzer/JoinNode.cpp
Normal file
159
src/Analyzer/JoinNode.cpp
Normal file
@ -0,0 +1,159 @@
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Analyzer/ListNode.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
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<ListNode>(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<const JoinNode &>(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<ASTTableJoin>();
|
||||
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<ASTTablesInSelectQuery>();
|
||||
|
||||
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<ASTTableJoin>();
|
||||
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<ASTTablesInSelectQueryElement &>();
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
148
src/Analyzer/JoinNode.h
Normal file
148
src/Analyzer/JoinNode.h
Normal file
@ -0,0 +1,148 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Joins.h>
|
||||
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
|
||||
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<JoinNode>;
|
||||
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -16,6 +16,12 @@ using ListNodePtr = std::shared_ptr<ListNode>;
|
||||
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
|
||||
{
|
||||
|
@ -178,6 +178,8 @@ String MatcherNode::getName() const
|
||||
}
|
||||
|
||||
buffer << ')';
|
||||
/// TODO: Transformers
|
||||
|
||||
return buffer.str();
|
||||
}
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -1,16 +1,22 @@
|
||||
#include <Analyzer/QueryNode.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -22,6 +28,29 @@ QueryNode::QueryNode()
|
||||
children[projection_child_index] = std::make_shared<ListNode>();
|
||||
}
|
||||
|
||||
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<ASTTableExpression>();
|
||||
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<ASTTablesInSelectQueryElement>();
|
||||
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<ASTTablesInSelectQuery>();
|
||||
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<ASTTablesInSelectQuery>();
|
||||
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())
|
||||
|
@ -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;
|
||||
|
@ -31,6 +31,8 @@
|
||||
#include <Analyzer/TableNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Analyzer/QueryNode.h>
|
||||
#include <Analyzer/ArrayJoinNode.h>
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Analyzer/InDepthQueryTreeVisitor.h>
|
||||
|
||||
#include <Databases/IDatabase.h>
|
||||
@ -360,8 +362,8 @@ QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_q
|
||||
}
|
||||
|
||||
auto & tables = tables_in_select_query->as<ASTTablesInSelectQuery &>();
|
||||
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());
|
||||
}
|
||||
|
||||
// if (table_element.table_join)
|
||||
// {
|
||||
// const auto & table_join = table_element.table_join->as<ASTTableJoin>();
|
||||
|
||||
// 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 join_expression = JoinExpression::create();
|
||||
// join_expression->getLeftTableExpression() = left_table_expression;
|
||||
// join_expression->getRightTableExpression() = right_table_expression;
|
||||
|
||||
// if (table_join->using_expression_list)
|
||||
// join_expression->getUsingExpressions() = getExpressionElements(table_join->using_expression_list, scope);
|
||||
|
||||
// if (table_join->on_expression)
|
||||
// {
|
||||
// join_expression->getOnExpression() = getExpressionElement(table_join->on_expression, scope);
|
||||
// }
|
||||
|
||||
// table_expressions.emplace_back(std::move(join_expression));
|
||||
// }
|
||||
|
||||
// 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());
|
||||
|
||||
// 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);
|
||||
|
||||
// table_expressions.push_back(array_join_expression);
|
||||
// }
|
||||
}
|
||||
|
||||
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_element.table_join)
|
||||
{
|
||||
const auto & table_join = table_element.table_join->as<ASTTableJoin &>();
|
||||
|
||||
// return table_expressions.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();
|
||||
|
||||
QueryTreeNodePtr join_expression;
|
||||
|
||||
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);
|
||||
|
||||
auto join_node = std::make_shared<JoinNode>(std::move(left_table_expression),
|
||||
std::move(right_table_expression),
|
||||
std::move(join_expression),
|
||||
table_join.locality,
|
||||
table_join.strictness,
|
||||
table_join.kind);
|
||||
|
||||
/** 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_expression = table_element.array_join->as<ASTArrayJoin &>();
|
||||
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 array_join_expressions_list = getExpressionList(array_join_expression.expression_list);
|
||||
|
||||
auto array_join_node = std::make_shared<ArrayJoinNode>(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));
|
||||
}
|
||||
}
|
||||
|
||||
if (table_expressions.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot be empty");
|
||||
|
||||
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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
@ -26,8 +26,8 @@ using TableNodePtr = std::shared_ptr<TableNode>;
|
||||
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;
|
||||
};
|
||||
|
||||
|
@ -1,5 +1,12 @@
|
||||
#include <Analyzer/Utils.h>
|
||||
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Analyzer/IdentifierNode.h>
|
||||
|
||||
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<IdentifierNode &>();
|
||||
const auto & identifier = identifier_node.getIdentifier();
|
||||
|
||||
if (identifier.getPartsSize() == 1)
|
||||
table_expression_node_ast = std::make_shared<ASTTableIdentifier>(identifier[0]);
|
||||
else if (identifier.getPartsSize() == 2)
|
||||
table_expression_node_ast = std::make_shared<ASTTableIdentifier>(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<ASTTableExpression>();
|
||||
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<ASTTablesInSelectQueryElement>();
|
||||
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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include <iostream>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
(void)(argc);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/TableOverrideUtils.h>
|
||||
@ -417,8 +418,17 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
|
||||
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
|
||||
QueryPlan 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<QueryPipelineSettings>(ast.getSettings());
|
||||
QueryPlan 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()));
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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;
|
||||
|
@ -46,51 +46,49 @@ namespace
|
||||
std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJoin::NameToTypeMap & source)
|
||||
{
|
||||
std::vector<std::string> 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 <typename SideTag = BothSidesTag, typename OnExpr, typename Func>
|
||||
bool forAllKeys(OnExpr & expressions, Func callback)
|
||||
template <typename SideTag, typename JoinClauses, typename Func>
|
||||
bool forKeyNamesInJoinClauses(JoinClauses & join_clauses, Func callback)
|
||||
{
|
||||
static_assert(std::is_same_v<SideTag, BothSidesTag> ||
|
||||
std::is_same_v<SideTag, LeftSideTag> ||
|
||||
std::is_same_v<SideTag, RightSideTag>);
|
||||
|
||||
for (auto & expr : expressions)
|
||||
for (auto & join_clause : join_clauses)
|
||||
{
|
||||
if constexpr (std::is_same_v<SideTag, BothSidesTag>)
|
||||
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<SideTag, RightSideTag> ? 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<SideTag, RightSideTag> ? 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<SideTag, BothSidesTag>)
|
||||
cont = callback(expr.key_names_left[i], expr.key_names_right[i]);
|
||||
if constexpr (std::is_same_v<SideTag, LeftSideTag>)
|
||||
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<SideTag, LeftSideTag>)
|
||||
should_continue = callback(join_clause.key_names_left[i]);
|
||||
if constexpr (std::is_same_v<SideTag, RightSideTag>)
|
||||
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<RightSideTag>(clauses, [this, &required](const auto & name)
|
||||
NameSet required_right_column_names;
|
||||
forKeyNamesInJoinClauses<RightSideTag>(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<BothSidesTag>(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<String> 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<DictionaryReader>(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<BothSidesTag>(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<BothSidesTag>(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<String, String> TableJoin::leftToRightKeyRemap() const
|
||||
if (hasUsing())
|
||||
{
|
||||
const auto & required_right_keys = requiredRightKeys();
|
||||
forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name)
|
||||
forKeyNamesInJoinClauses<BothSidesTag>(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<LeftSideTag>(clauses, func);
|
||||
forKeyNamesInJoinClauses<LeftSideTag>(clauses, func);
|
||||
else
|
||||
forAllKeys<RightSideTag>(clauses, func);
|
||||
forKeyNamesInJoinClauses<RightSideTag>(clauses, func);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void TableJoin::assertHasOneOnExpr() const
|
||||
void TableJoin::assertHasSingleClause() const
|
||||
{
|
||||
if (!oneDisjunct())
|
||||
{
|
||||
|
@ -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<String, String> condColumnNames() const
|
||||
{
|
||||
std::pair<String, String> 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 <typename LeftNamesAndTypes, typename RightNamesAndTypes>
|
||||
@ -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<JoinOnClause> & getClauses() { return clauses; }
|
||||
const std::vector<JoinOnClause> & 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 <typename TColumns>
|
||||
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; }
|
||||
|
@ -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<TableJoin> & 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<ASTTableJoin>() : nullptr;
|
||||
if (table_join_ast && tables_with_columns.size() >= 2)
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
133
tests/queries/0_stateless/02371_analyzer_join_cross.reference
Normal file
133
tests/queries/0_stateless/02371_analyzer_join_cross.reference
Normal file
@ -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
|
78
tests/queries/0_stateless/02371_analyzer_join_cross.sql
Normal file
78
tests/queries/0_stateless/02371_analyzer_join_cross.sql
Normal file
@ -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;
|
@ -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
|
156
tests/queries/0_stateless/02372_analyzer_join_inner.sql
Normal file
156
tests/queries/0_stateless/02372_analyzer_join_inner.sql
Normal file
@ -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;
|
Loading…
Reference in New Issue
Block a user