Added support for JOIN, ARRAY JOIN

This commit is contained in:
Maksim Kita 2022-08-15 18:34:10 +02:00
parent 1717d16fa4
commit 40cb904a3d
34 changed files with 3638 additions and 825 deletions

View 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;
}
}

View 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;
};
}

View File

@ -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);
}

View File

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

View File

@ -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";
}
}

View File

@ -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
View 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
View 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;
};
}

View File

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

View File

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

View File

@ -178,6 +178,8 @@ String MatcherNode::getName() const
}
buffer << ')';
/// TODO: Transformers
return buffer.str();
}

File diff suppressed because it is too large Load Diff

View File

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

View File

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

View File

@ -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();
}

View File

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

View File

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

View File

@ -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());
}
}
}
}

View File

@ -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);
}

View File

@ -1,7 +1,5 @@
#include <iostream>
using namespace DB;
int main(int argc, char ** argv)
{
(void)(argc);

View File

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

View File

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

View File

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

View File

@ -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())
{

View File

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

View File

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

View File

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

View File

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

View File

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

View 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

View 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;

View File

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

View 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;